From 2d0150c1a2688296346fa279b1f8d14edac935eb Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Sat, 5 Apr 2014 15:17:50 -0700 Subject: [PATCH 001/641] Remove the getStageInfo() method from SparkContext. This method exposes the Stage objects, which are private to Spark and should not be exposed to the user. This method was added in https://github.com/apache/spark/commit/01d77f329f5878b7c8672bbdc1859f3ca95d759d; ccing @squito here in case there's a good reason to keep this! Author: Kay Ousterhout Closes #308 from kayousterhout/remove_public_method and squashes the following commits: 2e2f009 [Kay Ousterhout] Remove the getStageInfo() method from SparkContext. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 4 ---- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index fcf16ce1b278e..8382dd44f3484 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -731,10 +731,6 @@ class SparkContext( */ def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap - def getStageInfo: Map[Stage, StageInfo] = { - dagScheduler.stageToInfos - } - /** * Return information about blocks stored in all of the slaves */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ef3d24d746829..442a95bb2c44b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -86,7 +86,7 @@ class DAGScheduler( private[scheduler] val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] private[scheduler] val jobIdToActiveJob = new HashMap[Int, ActiveJob] private[scheduler] val resultStageToJob = new HashMap[Stage, ActiveJob] - private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] + private[scheduler] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] // Stages we need to run whose parents aren't done private[scheduler] val waitingStages = new HashSet[Stage] From 6e88583aef7d8caf59d53c9fcb659a62d2cd6051 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Sat, 5 Apr 2014 15:23:37 -0700 Subject: [PATCH 002/641] [SPARK-1371] fix computePreferredLocations signature to not depend on underlying implementation Change to Map and Set - not mutable HashMap and HashSet Author: Mridul Muralidharan Closes #302 from mridulm/master and squashes the following commits: df747af [Mridul Muralidharan] Address review comments 17e2907 [Mridul Muralidharan] fix computePreferredLocations signature to not depend on underlying implementation --- .../scala/org/apache/spark/scheduler/InputFormatInfo.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index 5555585c8b4cd..b3f2cb346f7da 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -164,8 +164,7 @@ object InputFormatInfo { PS: I know the wording here is weird, hopefully it makes some sense ! */ - def computePreferredLocations(formats: Seq[InputFormatInfo]): HashMap[String, HashSet[SplitInfo]] - = { + def computePreferredLocations(formats: Seq[InputFormatInfo]): Map[String, Set[SplitInfo]] = { val nodeToSplit = new HashMap[String, HashSet[SplitInfo]] for (inputSplit <- formats) { @@ -178,6 +177,6 @@ object InputFormatInfo { } } - nodeToSplit + nodeToSplit.mapValues(_.toSet).toMap } } From 890d63bd4e16296ac70e151b3754727ea42b583c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 5 Apr 2014 19:08:24 -0700 Subject: [PATCH 003/641] Fix for PR #195 for Java 6 Use Java 6's recommended equivalent of Java 7's Logger.getGlobal() to retain Java 6 compatibility. See PR #195 Author: Sean Owen Closes #334 from srowen/FixPR195ForJava6 and squashes the following commits: f92fbd3 [Sean Owen] Use Java 6's recommended equivalent of Java 7's Logger.getGlobal() to retain Java 6 compatibility --- .../scala/org/apache/spark/sql/parquet/ParquetRelation.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 114bfbb719ee9..505ad0a2c77c1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -103,7 +103,7 @@ private[sql] object ParquetRelation { SLF4JBridgeHandler.install() for(name <- loggerNames) { val logger = Logger.getLogger(name) - logger.setParent(Logger.getGlobal) + logger.setParent(Logger.getLogger(Logger.GLOBAL_LOGGER_NAME)) logger.setUseParentHandlers(true) } } From 0b855167818b9afd2d2aa9f617b9861d77b2425d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 5 Apr 2014 20:52:05 -0700 Subject: [PATCH 004/641] SPARK-1421. Make MLlib work on Python 2.6 The reason it wasn't working was passing a bytearray to stream.write(), which is not supported in Python 2.6 but is in 2.7. (This array came from NumPy when we converted data to send it over to Java). Now we just convert those bytearrays to strings of bytes, which preserves nonprintable characters as well. Author: Matei Zaharia Closes #335 from mateiz/mllib-python-2.6 and squashes the following commits: f26c59f [Matei Zaharia] Update docs to no longer say we need Python 2.7 a84d6af [Matei Zaharia] SPARK-1421. Make MLlib work on Python 2.6 --- docs/mllib-guide.md | 3 +-- docs/python-programming-guide.md | 2 +- python/pyspark/mllib/__init__.py | 6 +----- python/pyspark/serializers.py | 11 ++++++++++- 4 files changed, 13 insertions(+), 9 deletions(-) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 203d235bf9663..a5e0cc50809cf 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -38,6 +38,5 @@ depends on native Fortran routines. You may need to install the if it is not already present on your nodes. MLlib will throw a linking error if it cannot detect these libraries automatically. -To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.7 or newer -and Python 2.7. +To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.7 or newer. diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index cbe7d820b455e..c2e5327324898 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -152,7 +152,7 @@ Many of the methods also contain [doctests](http://docs.python.org/2/library/doc # Libraries [MLlib](mllib-guide.html) is also available in PySpark. To use it, you'll need -[NumPy](http://www.numpy.org) version 1.7 or newer, and Python 2.7. The [MLlib guide](mllib-guide.html) contains +[NumPy](http://www.numpy.org) version 1.7 or newer. The [MLlib guide](mllib-guide.html) contains some example applications. # Where to Go from Here diff --git a/python/pyspark/mllib/__init__.py b/python/pyspark/mllib/__init__.py index b420d7a7f23ba..538ff26ce7c33 100644 --- a/python/pyspark/mllib/__init__.py +++ b/python/pyspark/mllib/__init__.py @@ -19,11 +19,7 @@ Python bindings for MLlib. """ -# MLlib currently needs Python 2.7+ and NumPy 1.7+, so complain if lower - -import sys -if sys.version_info[0:2] < (2, 7): - raise Exception("MLlib requires Python 2.7+") +# MLlib currently needs and NumPy 1.7+, so complain if lower import numpy if numpy.version.version < '1.7': diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 4d802924df4a1..b253807974a2e 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -64,6 +64,7 @@ from itertools import chain, izip, product import marshal import struct +import sys from pyspark import cloudpickle @@ -113,6 +114,11 @@ class FramedSerializer(Serializer): where C{length} is a 32-bit integer and data is C{length} bytes. """ + def __init__(self): + # On Python 2.6, we can't write bytearrays to streams, so we need to convert them + # to strings first. Check if the version number is that old. + self._only_write_strings = sys.version_info[0:2] <= (2, 6) + def dump_stream(self, iterator, stream): for obj in iterator: self._write_with_length(obj, stream) @@ -127,7 +133,10 @@ def load_stream(self, stream): def _write_with_length(self, obj, stream): serialized = self.dumps(obj) write_int(len(serialized), stream) - stream.write(serialized) + if self._only_write_strings: + stream.write(str(serialized)) + else: + stream.write(serialized) def _read_with_length(self, stream): length = read_int(stream) From 7012ffafad8fa876aa8bcb0b848445eec6734ef1 Mon Sep 17 00:00:00 2001 From: witgo Date: Sun, 6 Apr 2014 16:03:06 -0700 Subject: [PATCH 005/641] Fix SPARK-1420 The maven build error for Spark Catalyst Author: witgo Closes #333 from witgo/SPARK-1420 and squashes the following commits: 902519e [witgo] add dependency scala-reflect to catalyst --- sql/catalyst/pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 0edce55a93338..9d5c6a857bb00 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -44,6 +44,10 @@ + + org.scala-lang + scala-reflect + org.apache.spark spark-core_${scala.binary.version} From e258e5040fa1905a04efcb7b3ca4a6d33e18fa61 Mon Sep 17 00:00:00 2001 From: Egor Pakhomov Date: Sun, 6 Apr 2014 16:41:23 -0700 Subject: [PATCH 006/641] [SPARK-1259] Make RDD locally iterable Author: Egor Pakhomov Closes #156 from epahomov/SPARK-1259 and squashes the following commits: 8ec8f24 [Egor Pakhomov] Make to local iterator shorter 34aa300 [Egor Pakhomov] Fix toLocalIterator docs 08363ef [Egor Pakhomov] SPARK-1259 from toLocallyIterable to toLocalIterator 6a994eb [Egor Pakhomov] SPARK-1259 Make RDD locally iterable 8be3dcf [Egor Pakhomov] SPARK-1259 Make RDD locally iterable 33ecb17 [Egor Pakhomov] SPARK-1259 Make RDD locally iterable --- .../org/apache/spark/api/java/JavaRDDLike.scala | 14 +++++++++++++- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 12 ++++++++++++ .../test/java/org/apache/spark/JavaAPISuite.java | 9 +++++++++ .../test/scala/org/apache/spark/rdd/RDDSuite.scala | 1 + 4 files changed, 35 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 e03b8e78d5f52..6e8ec8e0c7629 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 @@ -17,7 +17,8 @@ package org.apache.spark.api.java -import java.util.{Comparator, List => JList} +import java.util.{Comparator, Iterator => JIterator, List => JList} +import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -280,6 +281,17 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { new java.util.ArrayList(arr) } + /** + * Return an iterator that contains all of the elements in this RDD. + * + * The iterator will consume as much memory as the largest partition in this RDD. + */ + def toLocalIterator(): JIterator[T] = { + import scala.collection.JavaConversions._ + rdd.toLocalIterator + } + + /** * 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 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 08c42c5ee87b6..c43823bd769b7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -661,6 +661,18 @@ abstract class RDD[T: ClassTag]( Array.concat(results: _*) } + /** + * Return an iterator that contains all of the elements in this RDD. + * + * The iterator will consume as much memory as the largest partition in this RDD. + */ + def toLocalIterator: Iterator[T] = { + def collectPartition(p: Int): Array[T] = { + sc.runJob(this, (iter: Iterator[T]) => iter.toArray, Seq(p), allowLocal = false).head + } + (0 until partitions.length).iterator.flatMap(i => collectPartition(i)) + } + /** * Return an array that contains all of the elements in this RDD. */ diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 2372f2d9924a1..762405be2a8f9 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -22,6 +22,7 @@ import scala.Tuple2; +import com.google.common.collect.Lists; import com.google.common.base.Optional; import com.google.common.base.Charsets; import com.google.common.io.Files; @@ -179,6 +180,14 @@ public void call(String s) { Assert.assertEquals(2, foreachCalls); } + @Test + public void toLocalIterator() { + List correct = Arrays.asList(1, 2, 3, 4); + JavaRDD rdd = sc.parallelize(correct); + List result = Lists.newArrayList(rdd.toLocalIterator()); + Assert.assertTrue(correct.equals(result)); + } + @SuppressWarnings("unchecked") @Test public void lookup() { 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 d6b5fdc7984b4..25973348a7837 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -33,6 +33,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("basic operations") { val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) assert(nums.collect().toList === List(1, 2, 3, 4)) + assert(nums.toLocalIterator.toList === List(1, 2, 3, 4)) val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2) assert(dups.distinct().count() === 4) assert(dups.distinct.count === 4) // Can distinct and count be called without parentheses? From 856c50f59bffbf76ad495eaab837febaf65cf02d Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 6 Apr 2014 17:40:37 -0700 Subject: [PATCH 007/641] SPARK-1387. Update build plugins, avoid plugin version warning, centralize versions Another handful of small build changes to organize and standardize a bit, and avoid warnings: - Update Maven plugin versions for good measure - Since plugins need maven 3.0.4 already, require it explicitly (<3.0.4 had some bugs anyway) - Use variables to define versions across dependencies where they should move in lock step - ... and make this consistent between Maven/SBT OK, I also updated the JIRA URL while I was at it here. Author: Sean Owen Closes #291 from srowen/SPARK-1387 and squashes the following commits: 461eca1 [Sean Owen] Couldn't resist also updating JIRA location to new one c2d5cc5 [Sean Owen] Update plugins and Maven version; use variables consistently across Maven/SBT to define dependency versions that should stay in step. --- assembly/pom.xml | 2 +- core/pom.xml | 2 - dev/audit-release/maven_app_core/pom.xml | 2 +- docs/building-with-maven.md | 2 +- examples/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 43 +++++++++++---------- project/SparkBuild.scala | 49 ++++++++++++++---------- streaming/pom.xml | 1 - 10 files changed, 57 insertions(+), 50 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index b5e752c6cd1f6..255107a2c47cb 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -208,7 +208,7 @@ org.codehaus.mojo buildnumber-maven-plugin - 1.1 + 1.2 validate diff --git a/core/pom.xml b/core/pom.xml index 66f9fc4961b03..1f808380817c9 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -117,12 +117,10 @@ com.twitter chill_${scala.binary.version} - 0.3.1 com.twitter chill-java - 0.3.1 commons-net diff --git a/dev/audit-release/maven_app_core/pom.xml b/dev/audit-release/maven_app_core/pom.xml index 0b837c01751fe..76a381f8e17e0 100644 --- a/dev/audit-release/maven_app_core/pom.xml +++ b/dev/audit-release/maven_app_core/pom.xml @@ -49,7 +49,7 @@ maven-compiler-plugin - 2.3.2 + 3.1 diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 730a6e7932564..9cebaf12283fc 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -6,7 +6,7 @@ title: Building Spark with Maven * This will become a table of contents (this text will be scraped). {:toc} -Building Spark using Maven Requires Maven 3 (the build process is tested with Maven 3.0.4) and Java 1.6 or newer. +Building Spark using Maven requires Maven 3.0.4 or newer and Java 1.6 or newer. ## Setting up Maven's Memory Usage ## diff --git a/examples/pom.xml b/examples/pom.xml index a5569ff5e71f3..0b6212b5d1549 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -110,7 +110,7 @@ org.apache.hbase hbase - 0.94.6 + ${hbase.version} asm diff --git a/graphx/pom.xml b/graphx/pom.xml index 5a5022916d234..b4c67ddcd8ca9 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -54,7 +54,7 @@ org.jblas jblas - 1.2.3 + ${jblas.version} org.eclipse.jetty diff --git a/mllib/pom.xml b/mllib/pom.xml index fec1cc94b2642..e7ce00efc4af6 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -58,7 +58,7 @@ org.jblas jblas - 1.2.3 + ${jblas.version} org.scalanlp diff --git a/pom.xml b/pom.xml index 01341d21b7f23..1426e0e00214c 100644 --- a/pom.xml +++ b/pom.xml @@ -54,11 +54,11 @@ JIRA - https://spark-project.atlassian.net/browse/SPARK + https://issues.apache.org/jira/browse/SPARK - 3.0.0 + 3.0.4 @@ -123,6 +123,10 @@ 0.94.6 0.12.0 1.3.2 + 1.2.3 + 8.1.14.v20131031 + 0.3.1 + 3.0.0 64m 512m @@ -192,22 +196,22 @@ org.eclipse.jetty jetty-util - 8.1.14.v20131031 + ${jetty.version} org.eclipse.jetty jetty-security - 8.1.14.v20131031 + ${jetty.version} org.eclipse.jetty jetty-plus - 8.1.14.v20131031 + ${jetty.version} org.eclipse.jetty jetty-server - 8.1.14.v20131031 + ${jetty.version} com.google.guava @@ -273,7 +277,7 @@ com.twitter chill_${scala.binary.version} - 0.3.1 + ${chill.version} org.ow2.asm @@ -288,7 +292,7 @@ com.twitter chill-java - 0.3.1 + ${chill.version} org.ow2.asm @@ -392,27 +396,27 @@ com.codahale.metrics metrics-core - 3.0.0 + ${codahale.metrics.version} com.codahale.metrics metrics-jvm - 3.0.0 + ${codahale.metrics.version} com.codahale.metrics metrics-json - 3.0.0 + ${codahale.metrics.version} com.codahale.metrics metrics-ganglia - 3.0.0 + ${codahale.metrics.version} com.codahale.metrics metrics-graphite - 3.0.0 + ${codahale.metrics.version} org.scala-lang @@ -585,7 +589,7 @@ org.apache.maven.plugins maven-enforcer-plugin - 1.1.1 + 1.3.1 enforce-versions @@ -595,7 +599,7 @@ - 3.0.0 + 3.0.4 ${java.version} @@ -608,12 +612,12 @@ org.codehaus.mojo build-helper-maven-plugin - 1.7 + 1.8 net.alchim31.maven scala-maven-plugin - 3.1.5 + 3.1.6 scala-compile-first @@ -674,7 +678,7 @@ org.apache.maven.plugins maven-surefire-plugin - 2.12.4 + 2.17 true @@ -713,7 +717,7 @@ org.apache.maven.plugins maven-shade-plugin - 2.0 + 2.2 org.apache.maven.plugins @@ -810,7 +814,6 @@ org.apache.maven.plugins maven-jar-plugin - 2.4 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 843a874fbfdb0..3489b43d43f0d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -248,10 +248,10 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "io.netty" % "netty-all" % "4.0.17.Final", - "org.eclipse.jetty" % "jetty-server" % "8.1.14.v20131031", - "org.eclipse.jetty" % "jetty-util" % "8.1.14.v20131031", - "org.eclipse.jetty" % "jetty-plus" % "8.1.14.v20131031", - "org.eclipse.jetty" % "jetty-security" % "8.1.14.v20131031", + "org.eclipse.jetty" % "jetty-server" % jettyVersion, + "org.eclipse.jetty" % "jetty-util" % jettyVersion, + "org.eclipse.jetty" % "jetty-plus" % jettyVersion, + "org.eclipse.jetty" % "jetty-security" % jettyVersion, /** Workaround for SPARK-959. Dependency used by org.eclipse.jetty. Fixed in ivy 2.3.0. */ "org.eclipse.jetty.orbit" % "javax.servlet" % "3.0.0.v201112011016" artifacts Artifact("javax.servlet", "jar", "jar"), "org.scalatest" %% "scalatest" % "1.9.1" % "test", @@ -276,6 +276,13 @@ object SparkBuild extends Build { publishLocalBoth <<= Seq(publishLocal in MavenCompile, publishLocal).dependOn ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings ++ ScalaStyleSettings + val akkaVersion = "2.2.3-shaded-protobuf" + val chillVersion = "0.3.1" + val codahaleMetricsVersion = "3.0.0" + val jblasVersion = "1.2.3" + val jettyVersion = "8.1.14.v20131031" + val hiveVersion = "0.12.0" + val parquetVersion = "1.3.2" val slf4jVersion = "1.7.5" val excludeNetty = ExclusionRule(organization = "org.jboss.netty") @@ -309,9 +316,9 @@ object SparkBuild extends Build { "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407 "com.ning" % "compress-lzf" % "1.0.0", "org.xerial.snappy" % "snappy-java" % "1.0.5", - "org.spark-project.akka" %% "akka-remote" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), - "org.spark-project.akka" %% "akka-slf4j" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), - "org.spark-project.akka" %% "akka-testkit" % "2.2.3-shaded-protobuf" % "test", + "org.spark-project.akka" %% "akka-remote" % akkaVersion excludeAll(excludeNetty), + "org.spark-project.akka" %% "akka-slf4j" % akkaVersion excludeAll(excludeNetty), + "org.spark-project.akka" %% "akka-testkit" % akkaVersion % "test", "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", @@ -321,12 +328,12 @@ object SparkBuild extends Build { "org.apache.derby" % "derby" % "10.4.2.0" % "test", "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm), "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeNetty), - "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-graphite" % "3.0.0", - "com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm), - "com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm), + "com.codahale.metrics" % "metrics-core" % codahaleMetricsVersion, + "com.codahale.metrics" % "metrics-jvm" % codahaleMetricsVersion, + "com.codahale.metrics" % "metrics-json" % codahaleMetricsVersion, + "com.codahale.metrics" % "metrics-graphite" % codahaleMetricsVersion, + "com.twitter" %% "chill" % chillVersion excludeAll(excludeAsm), + "com.twitter" % "chill-java" % chillVersion excludeAll(excludeAsm), "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), "com.clearspring.analytics" % "stream" % "2.5.1" ), @@ -370,7 +377,7 @@ object SparkBuild extends Build { name := "spark-graphx", previousArtifact := sparkPreviousArtifact("spark-graphx"), libraryDependencies ++= Seq( - "org.jblas" % "jblas" % "1.2.3" + "org.jblas" % "jblas" % jblasVersion ) ) @@ -383,7 +390,7 @@ object SparkBuild extends Build { name := "spark-mllib", previousArtifact := sparkPreviousArtifact("spark-mllib"), libraryDependencies ++= Seq( - "org.jblas" % "jblas" % "1.2.3", + "org.jblas" % "jblas" % jblasVersion, "org.scalanlp" %% "breeze" % "0.7" ) ) @@ -403,8 +410,8 @@ object SparkBuild extends Build { def sqlCoreSettings = sharedSettings ++ Seq( name := "spark-sql", libraryDependencies ++= Seq( - "com.twitter" % "parquet-column" % "1.3.2", - "com.twitter" % "parquet-hadoop" % "1.3.2" + "com.twitter" % "parquet-column" % parquetVersion, + "com.twitter" % "parquet-hadoop" % parquetVersion ) ) @@ -416,9 +423,9 @@ object SparkBuild extends Build { jarName in packageDependency <<= version map { v => "spark-hive-assembly-" + v + "-hadoop" + hadoopVersion + "-deps.jar" }, javaOptions += "-XX:MaxPermSize=1g", libraryDependencies ++= Seq( - "org.apache.hive" % "hive-metastore" % "0.12.0", - "org.apache.hive" % "hive-exec" % "0.12.0", - "org.apache.hive" % "hive-serde" % "0.12.0" + "org.apache.hive" % "hive-metastore" % hiveVersion, + "org.apache.hive" % "hive-exec" % hiveVersion, + "org.apache.hive" % "hive-serde" % hiveVersion ), // Multiple queries rely on the TestHive singleton. See comments there for more details. parallelExecution in Test := false, @@ -549,7 +556,7 @@ object SparkBuild extends Build { name := "spark-streaming-zeromq", previousArtifact := sparkPreviousArtifact("spark-streaming-zeromq"), libraryDependencies ++= Seq( - "org.spark-project.akka" %% "akka-zeromq" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty) + "org.spark-project.akka" %% "akka-zeromq" % akkaVersion excludeAll(excludeNetty) ) ) diff --git a/streaming/pom.xml b/streaming/pom.xml index 1953cc6883378..93b1c5a37aff9 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -96,7 +96,6 @@ org.apache.maven.plugins maven-jar-plugin - 2.2 From 7ce52c4a7a07b0db5e7c1312b1920efb1165ce6a Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sun, 6 Apr 2014 17:43:44 -0700 Subject: [PATCH 008/641] SPARK-1349: spark-shell gets its own command history Currently, spark-shell shares its command history with scala repl. This fix is simply a modification of the default FileBackedHistory file setting: https://github.com/scala/scala/blob/master/src/repl/scala/tools/nsc/interpreter/session/FileBackedHistory.scala#L77 Author: Aaron Davidson Closes #267 from aarondav/repl and squashes the following commits: f9c62d2 [Aaron Davidson] SPARK-1349: spark-shell gets its own command history separate from scala repl --- .../org/apache/spark/repl/SparkJLineReader.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala b/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala index 946e71039088d..0db26c3407dff 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala @@ -7,8 +7,10 @@ package org.apache.spark.repl +import scala.reflect.io.{Path, File} import scala.tools.nsc._ import scala.tools.nsc.interpreter._ +import scala.tools.nsc.interpreter.session.JLineHistory.JLineFileHistory import scala.tools.jline.console.ConsoleReader import scala.tools.jline.console.completer._ @@ -25,7 +27,7 @@ class SparkJLineReader(_completion: => Completion) extends InteractiveReader { val consoleReader = new JLineConsoleReader() lazy val completion = _completion - lazy val history: JLineHistory = JLineHistory() + lazy val history: JLineHistory = new SparkJLineHistory private def term = consoleReader.getTerminal() def reset() = term.reset() @@ -78,3 +80,11 @@ class SparkJLineReader(_completion: => Completion) extends InteractiveReader { def readOneLine(prompt: String) = consoleReader readLine prompt def readOneKey(prompt: String) = consoleReader readOneKey prompt } + +/** Changes the default history file to not collide with the scala repl's. */ +class SparkJLineHistory extends JLineFileHistory { + import Properties.userHome + + def defaultFileName = ".spark_history" + override protected lazy val historyFile = File(Path(userHome) / defaultFileName) +} From 4106558435889261243d186f5f0b51c5f9e98d56 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sun, 6 Apr 2014 17:48:41 -0700 Subject: [PATCH 009/641] SPARK-1314: Use SPARK_HIVE to determine if we include Hive in packaging Previously, we based our decision regarding including datanucleus jars based on the existence of a spark-hive-assembly jar, which was incidentally built whenever "sbt assembly" is run. This means that a typical and previously supported pathway would start using hive jars. This patch has the following features/bug fixes: - Use of SPARK_HIVE (default false) to determine if we should include Hive in the assembly jar. - Analagous feature in Maven with -Phive (previously, there was no support for adding Hive to any of our jars produced by Maven) - assemble-deps fixed since we no longer use a different ASSEMBLY_DIR - avoid adding log message in compute-classpath.sh to the classpath :) Still TODO before mergeable: - We need to download the datanucleus jars outside of sbt. Perhaps we can have spark-class download them if SPARK_HIVE is set similar to how sbt downloads itself. - Spark SQL documentation updates. Author: Aaron Davidson Closes #237 from aarondav/master and squashes the following commits: 5dc4329 [Aaron Davidson] Typo fixes dd4f298 [Aaron Davidson] Doc update dd1a365 [Aaron Davidson] Eliminate need for SPARK_HIVE at runtime by d/ling datanucleus from Maven a9269b5 [Aaron Davidson] [WIP] Use SPARK_HIVE to determine if we include Hive in packaging --- assembly/pom.xml | 10 ++++++++ bin/compute-classpath.sh | 35 +++++++++++++++------------- bin/spark-class | 2 -- dev/create-release/create-release.sh | 4 ++-- docs/sql-programming-guide.md | 4 ++-- pom.xml | 7 +++++- project/SparkBuild.scala | 25 +++++++++++++------- sql/hive/pom.xml | 28 ++++++++++++++++++++++ 8 files changed, 83 insertions(+), 32 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 255107a2c47cb..923bf47f7076a 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -163,6 +163,16 @@ + + hive + + + org.apache.spark + spark-hive_${scala.binary.version} + ${project.version} + + + spark-ganglia-lgpl diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index bef42df71ce01..be37102dc069a 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -30,21 +30,7 @@ FWDIR="$(cd `dirname $0`/..; pwd)" # Build up classpath CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" -# Support for interacting with Hive. Since hive pulls in a lot of dependencies that might break -# existing Spark applications, it is not included in the standard spark assembly. Instead, we only -# include it in the classpath if the user has explicitly requested it by running "sbt hive/assembly" -# Hopefully we will find a way to avoid uber-jars entirely and deploy only the needed packages in -# the future. -if [ -f "$FWDIR"/sql/hive/target/scala-$SCALA_VERSION/spark-hive-assembly-*.jar ]; then - - # Datanucleus jars do not work if only included in the uberjar as plugin.xml metadata is lost. - DATANUCLEUSJARS=$(JARS=("$FWDIR/lib_managed/jars"/datanucleus-*.jar); IFS=:; echo "${JARS[*]}") - CLASSPATH=$CLASSPATH:$DATANUCLEUSJARS - - ASSEMBLY_DIR="$FWDIR/sql/hive/target/scala-$SCALA_VERSION/" -else - ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION/" -fi +ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION" # First check if we have a dependencies jar. If so, include binary classes with the deps jar if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then @@ -59,7 +45,7 @@ if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" - DEPS_ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark*-assembly*hadoop*-deps.jar` + DEPS_ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar` CLASSPATH="$CLASSPATH:$DEPS_ASSEMBLY_JAR" else # Else use spark-assembly jar from either RELEASE or assembly directory @@ -71,6 +57,23 @@ else CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" fi +# When Hive support is needed, Datanucleus jars must be included on the classpath. +# Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. +# Both sbt and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is +# built with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark +# assembly is built for Hive, before actually populating the CLASSPATH with the jars. +# Note that this check order is faster (by up to half a second) in the case where Hive is not used. +num_datanucleus_jars=$(ls "$FWDIR"/lib_managed/jars/ | grep "datanucleus-.*\\.jar" | wc -l) +if [ $num_datanucleus_jars -gt 0 ]; then + AN_ASSEMBLY_JAR=${ASSEMBLY_JAR:-$DEPS_ASSEMBLY_JAR} + num_hive_files=$(jar tvf "$AN_ASSEMBLY_JAR" org/apache/hadoop/hive/ql/exec 2>/dev/null | wc -l) + if [ $num_hive_files -gt 0 ]; then + echo "Spark assembly has been built with Hive, including Datanucleus jars on classpath" 1>&2 + DATANUCLEUSJARS=$(echo "$FWDIR/lib_managed/jars"/datanucleus-*.jar | tr " " :) + CLASSPATH=$CLASSPATH:$DATANUCLEUSJARS + fi +fi + # Add test classes if we're running from SBT or Maven with SPARK_TESTING set to 1 if [[ $SPARK_TESTING == 1 ]]; then CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/test-classes" diff --git a/bin/spark-class b/bin/spark-class index 0dcf0e156cb52..76fde3e448891 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -154,5 +154,3 @@ if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then fi exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" - - diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 995106f111443..bf1c5d7953bd2 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -49,14 +49,14 @@ mvn -DskipTests \ -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Pspark-ganglia-lgpl \ + -Pyarn -Phive -Pspark-ganglia-lgpl\ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ --batch-mode release:prepare mvn -DskipTests \ -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Pspark-ganglia-lgpl\ release:perform rm -rf spark diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index f849716f7a48f..a59393e1424de 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -264,8 +264,8 @@ evaluated by the SQL execution engine. A full list of the functions supported c Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). However, since Hive has a large number of dependencies, it is not included in the default Spark assembly. -In order to use Hive you must first run '`SPARK_HIVE=true sbt/sbt assembly/assembly`'. This command builds a new assembly -jar that includes Hive. Note that this Hive assembly jar must also be present +In order to use Hive you must first run '`SPARK_HIVE=true sbt/sbt assembly/assembly`' (or use `-Phive` for maven). +This command builds a new assembly jar that includes Hive. Note that this Hive assembly jar must also be present on all of the worker nodes, as they will need access to the Hive serialization and deserialization libraries (SerDes) in order to acccess data stored in Hive. diff --git a/pom.xml b/pom.xml index 1426e0e00214c..c03bb35c99442 100644 --- a/pom.xml +++ b/pom.xml @@ -377,7 +377,6 @@ org.apache.derby derby 10.4.2.0 - test net.liftweb @@ -580,6 +579,12 @@ + + + org.codehaus.jackson + jackson-mapper-asl + 1.8.8 + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 3489b43d43f0d..d1e4b8b964b88 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -43,6 +43,8 @@ object SparkBuild extends Build { val DEFAULT_YARN = false + val DEFAULT_HIVE = false + // HBase version; set as appropriate. val HBASE_VERSION = "0.94.6" @@ -67,15 +69,17 @@ object SparkBuild extends Build { lazy val sql = Project("sql", file("sql/core"), settings = sqlCoreSettings) dependsOn(core, catalyst) - // Since hive is its own assembly, it depends on all of the modules. - lazy val hive = Project("hive", file("sql/hive"), settings = hiveSettings) dependsOn(sql, graphx, bagel, mllib, streaming, repl) + lazy val hive = Project("hive", file("sql/hive"), settings = hiveSettings) dependsOn(sql) + + lazy val maybeHive: Seq[ClasspathDependency] = if (isHiveEnabled) Seq(hive) else Seq() + lazy val maybeHiveRef: Seq[ProjectReference] = if (isHiveEnabled) Seq(hive) else Seq() lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn(core) 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, streaming, repl, sql) dependsOn(maybeYarn: _*) dependsOn(maybeGanglia: _*) + .dependsOn(core, graphx, bagel, mllib, streaming, repl, sql) dependsOn(maybeYarn: _*) dependsOn(maybeHive: _*) dependsOn(maybeGanglia: _*) lazy val assembleDeps = TaskKey[Unit]("assemble-deps", "Build assembly of dependencies and packages Spark projects") @@ -101,6 +105,11 @@ 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() + lazy val isHiveEnabled = Properties.envOrNone("SPARK_HIVE") match { + case None => DEFAULT_HIVE + case Some(v) => v.toBoolean + } + // 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 @@ -141,13 +150,13 @@ object SparkBuild extends Build { lazy val allExternalRefs = Seq[ProjectReference](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) - .dependsOn(core, mllib, graphx, bagel, streaming, externalTwitter, hive) dependsOn(allExternal: _*) + .dependsOn(core, mllib, graphx, bagel, streaming, hive) dependsOn(allExternal: _*) // Everything except assembly, hive, tools, java8Tests and examples belong to packageProjects - lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx, catalyst, sql) ++ maybeYarnRef ++ maybeGangliaRef + lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx, catalyst, sql) ++ maybeYarnRef ++ maybeHiveRef ++ maybeGangliaRef lazy val allProjects = packageProjects ++ allExternalRefs ++ - Seq[ProjectReference](examples, tools, assemblyProj, hive) ++ maybeJava8Tests + Seq[ProjectReference](examples, tools, assemblyProj) ++ maybeJava8Tests def sharedSettings = Defaults.defaultSettings ++ MimaBuild.mimaSettings(file(sparkHome)) ++ Seq( organization := "org.apache.spark", @@ -417,10 +426,8 @@ object SparkBuild extends Build { // Since we don't include hive in the main assembly this project also acts as an alternative // assembly jar. - def hiveSettings = sharedSettings ++ assemblyProjSettings ++ Seq( + def hiveSettings = sharedSettings ++ Seq( name := "spark-hive", - jarName in assembly <<= version map { v => "spark-hive-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" }, - jarName in packageDependency <<= version map { v => "spark-hive-assembly-" + v + "-hadoop" + hadoopVersion + "-deps.jar" }, javaOptions += "-XX:MaxPermSize=1g", libraryDependencies ++= Seq( "org.apache.hive" % "hive-metastore" % hiveVersion, diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 63f592cb4b441..a662da76ce25a 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -63,6 +63,10 @@ hive-exec ${hive.version} + + org.codehaus.jackson + jackson-mapper-asl + org.apache.hive hive-serde @@ -87,6 +91,30 @@ org.scalatest scalatest-maven-plugin + + + + org.apache.maven.plugins + maven-dependency-plugin + 2.4 + + + copy-dependencies + package + + copy-dependencies + + + + ${basedir}/../../lib_managed/jars + false + false + true + org.datanucleus + + + + From 1440154c27ca48b5a75103eccc9057286d3f6ca8 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Sun, 6 Apr 2014 19:17:33 -0700 Subject: [PATCH 010/641] SPARK-1154: Clean up app folders in worker nodes This is a fix for [SPARK-1154](https://issues.apache.org/jira/browse/SPARK-1154). The issue is that worker nodes fill up with a huge number of app-* folders after some time. This change adds a periodic cleanup task which asynchronously deletes app directories older than a configurable TTL. Two new configuration parameters have been introduced: spark.worker.cleanup_interval spark.worker.app_data_ttl This change does not include moving the downloads of application jars to a location outside of the work directory. We will address that if we have time, but that potentially involves caching so it will come either as part of this PR or a separate PR. Author: Evan Chan Author: Kelvin Chu Closes #288 from velvia/SPARK-1154-cleanup-app-folders and squashes the following commits: 0689995 [Evan Chan] CR from @aarondav - move config, clarify for standalone mode 9f10d96 [Evan Chan] CR from @pwendell - rename configs and add cleanup.enabled f2f6027 [Evan Chan] CR from @andrewor14 553d8c2 [Kelvin Chu] change the variable name to currentTimeMillis since it actually tracks in seconds 8dc9cb5 [Kelvin Chu] Fixed a bug in Utils.findOldFiles() after merge. cb52f2b [Kelvin Chu] Change the name of findOldestFiles() to findOldFiles() 72f7d2d [Kelvin Chu] Fix a bug of Utils.findOldestFiles(). file.lastModified is returned in milliseconds. ad99955 [Kelvin Chu] Add unit test for Utils.findOldestFiles() dc1a311 [Evan Chan] Don't recompute current time with every new file e3c408e [Evan Chan] Document the two new settings b92752b [Evan Chan] SPARK-1154: Add a periodic task to clean up app directories --- .../apache/spark/deploy/DeployMessage.scala | 4 +++ .../apache/spark/deploy/worker/Worker.scala | 23 +++++++++++++++- .../scala/org/apache/spark/util/Utils.scala | 19 ++++++++++++-- .../org/apache/spark/util/UtilsSuite.scala | 15 ++++++++++- docs/configuration.md | 26 +++++++++++++++++++ 5 files changed, 83 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 83ce14a0a806a..a7368f9f3dfbe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -86,6 +86,10 @@ private[deploy] object DeployMessages { case class KillDriver(driverId: String) extends DeployMessage + // Worker internal + + case object WorkDirCleanup // Sent to Worker actor periodically for cleaning up app folders + // AppClient to Master case class RegisterApplication(appDescription: ApplicationDescription) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 8a71ddda4cb5e..bf5a8d09dd2df 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -64,6 +64,12 @@ private[spark] class Worker( val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 + val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", true) + // How often worker will clean up old app folders + val CLEANUP_INTERVAL_MILLIS = conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000 + // TTL for app folders/data; after TTL expires it will be cleaned up + val APP_DATA_RETENTION_SECS = conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) + // Index into masterUrls that we're currently trying to register with. var masterIndex = 0 @@ -179,12 +185,28 @@ private[spark] class Worker( registered = true changeMaster(masterUrl, masterWebUiUrl) context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis, self, SendHeartbeat) + if (CLEANUP_ENABLED) { + context.system.scheduler.schedule(CLEANUP_INTERVAL_MILLIS millis, + CLEANUP_INTERVAL_MILLIS millis, self, WorkDirCleanup) + } case SendHeartbeat => masterLock.synchronized { if (connected) { master ! Heartbeat(workerId) } } + case WorkDirCleanup => + // Spin up a separate thread (in a future) to do the dir cleanup; don't tie up worker actor + val cleanupFuture = concurrent.future { + logInfo("Cleaning up oldest application directories in " + workDir + " ...") + Utils.findOldFiles(workDir, APP_DATA_RETENTION_SECS) + .foreach(Utils.deleteRecursively) + } + cleanupFuture onFailure { + case e: Throwable => + logError("App dir cleanup failed: " + e.getMessage, e) + } + case MasterChanged(masterUrl, masterWebUiUrl) => logInfo("Master has changed, new master is at " + masterUrl) changeMaster(masterUrl, masterWebUiUrl) @@ -331,7 +353,6 @@ private[spark] class Worker( } private[spark] object Worker { - def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index d3c39dee330b2..4435b21a7505e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -597,9 +597,24 @@ private[spark] object Utils extends Logging { } if (fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile())) { - return false; + return false } else { - return true; + return true + } + } + + /** + * Finds all the files in a directory whose last modified time is older than cutoff seconds. + * @param dir must be the path to a directory, or IllegalArgumentException is thrown + * @param cutoff measured in seconds. Files older than this are returned. + */ + def findOldFiles(dir: File, cutoff: Long): Seq[File] = { + val currentTimeMillis = System.currentTimeMillis + if (dir.isDirectory) { + val files = listFilesSafely(dir) + files.filter { file => file.lastModified < (currentTimeMillis - cutoff * 1000) } + } else { + throw new IllegalArgumentException(dir + " is not a directory!") } } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 616214fb5e3a6..eb7fb6318262b 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.util import scala.util.Random -import java.io.{ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} +import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} import java.nio.{ByteBuffer, ByteOrder} import com.google.common.base.Charsets @@ -154,5 +154,18 @@ class UtilsSuite extends FunSuite { val iterator = Iterator.range(0, 5) assert(Utils.getIteratorSize(iterator) === 5L) } + + test("findOldFiles") { + // create some temporary directories and files + val parent: File = Utils.createTempDir() + val child1: File = Utils.createTempDir(parent.getCanonicalPath) // The parent directory has two child directories + val child2: File = Utils.createTempDir(parent.getCanonicalPath) + // set the last modified time of child1 to 10 secs old + child1.setLastModified(System.currentTimeMillis() - (1000 * 10)) + + val result = Utils.findOldFiles(parent, 5) // find files older than 5 secs + assert(result.size.equals(1)) + assert(result(0).getCanonicalPath.equals(child1.getCanonicalPath)) + } } diff --git a/docs/configuration.md b/docs/configuration.md index b6005acac8b93..57bda20edcdf1 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -348,6 +348,32 @@ Apart from these, the following properties are also available, and may be useful receives no heartbeats. + + spark.worker.cleanup.enabled + true + + Enable periodic cleanup of worker / application directories. Note that this only affects standalone + mode, as YARN works differently. + + + + spark.worker.cleanup.interval + 1800 (30 minutes) + + Controls the interval, in seconds, at which the worker cleans up old application work dirs + on the local machine. + + + + spark.worker.cleanup.appDataTtl + 7 * 24 * 3600 (7 days) + + The number of seconds to retain application work directories on each worker. This is a Time To Live + and should depend on the amount of available disk space you have. Application logs and jars are + downloaded to each application work dir. Over time, the work dirs can quickly fill up disk space, + especially if you run jobs very frequently. + + spark.akka.frameSize 10 From 87d0928a3301835705652c24a26096546597e156 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 6 Apr 2014 21:04:45 -0700 Subject: [PATCH 011/641] SPARK-1431: Allow merging conflicting pull requests Sometimes if there is a small conflict it's nice to be able to just manually fix it up rather than have another RTT with the contributor. Author: Patrick Wendell Closes #342 from pwendell/merge-conflicts and squashes the following commits: cdce61a [Patrick Wendell] SPARK-1431: Allow merging conflicting pull requests --- dev/merge_spark_pr.py | 26 ++++++++++++++++++++++---- 1 file changed, 22 insertions(+), 4 deletions(-) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index e8f78fc5f231a..7a61943e94814 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -87,11 +87,20 @@ def merge_pr(pr_num, target_ref): run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, target_ref, target_branch_name)) run_cmd("git checkout %s" % target_branch_name) - run_cmd(['git', 'merge', pr_branch_name, '--squash']) + had_conflicts = False + try: + run_cmd(['git', 'merge', pr_branch_name, '--squash']) + except Exception as e: + msg = "Error merging: %s\nWould you like to manually fix-up this merge?" % e + continue_maybe(msg) + msg = "Okay, please fix any conflicts and 'git add' conflicting files... Finished?" + continue_maybe(msg) + had_conflicts = True commit_authors = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, '--pretty=format:%an <%ae>']).split("\n") - distinct_authors = sorted(set(commit_authors), key=lambda x: commit_authors.count(x), reverse=True) + distinct_authors = sorted(set(commit_authors), key=lambda x: commit_authors.count(x), + reverse=True) primary_author = distinct_authors[0] commits = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, '--pretty=format:%h [%an] %s']).split("\n\n") @@ -105,6 +114,13 @@ def merge_pr(pr_num, target_ref): merge_message_flags += ["-m", authors] + if had_conflicts: + committer_name = run_cmd("git config --get user.name").strip() + committer_email = run_cmd("git config --get user.email").strip() + message = "This patch had conflicts when merged, resolved by\nCommitter: %s <%s>" % ( + committer_name, committer_email) + merge_message_flags += ["-m", message] + # The string "Closes #%s" string is required for GitHub to correctly close the PR merge_message_flags += ["-m", "Closes #%s from %s and squashes the following commits:" % (pr_num, pr_repo_desc)] @@ -186,8 +202,10 @@ def maybe_cherry_pick(pr_num, merge_hash, default_branch): maybe_cherry_pick(pr_num, merge_hash, latest_branch) sys.exit(0) -if bool(pr["mergeable"]) == False: - fail("Pull request %s is not mergeable in its current form" % pr_num) +if not bool(pr["mergeable"]): + msg = "Pull request %s is not mergeable in its current form.\n" % pr_num + \ + "Continue? (experts only!)" + continue_maybe(msg) print ("\n=== Pull Request #%s ===" % pr_num) print("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( From accd0999f9cb6a449434d3fc5274dd469eeecab2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 7 Apr 2014 00:14:00 -0700 Subject: [PATCH 012/641] [SQL] SPARK-1371 Hash Aggregation Improvements Given: ```scala case class Data(a: Int, b: Int) val rdd = sparkContext .parallelize(1 to 200) .flatMap(_ => (1 to 50000).map(i => Data(i % 100, i))) rdd.registerAsTable("data") cacheTable("data") ``` Before: ``` SELECT COUNT(*) FROM data:[10000000] 16795.567ms SELECT a, SUM(b) FROM data GROUP BY a 7536.436ms SELECT SUM(b) FROM data 10954.1ms ``` After: ``` SELECT COUNT(*) FROM data:[10000000] 1372.175ms SELECT a, SUM(b) FROM data GROUP BY a 2070.446ms SELECT SUM(b) FROM data 958.969ms ``` Author: Michael Armbrust Closes #295 from marmbrus/hashAgg and squashes the following commits: ec63575 [Michael Armbrust] Add comment. d0495a9 [Michael Armbrust] Use scaladoc instead. b4a6887 [Michael Armbrust] Address review comments. a2d90ba [Michael Armbrust] Capture child output statically to avoid issues with generators and serialization. 7c13112 [Michael Armbrust] Rewrite Aggregate operator to stream input and use projections. Remove unused local RDD functions implicits. 5096f99 [Michael Armbrust] Make HiveUDAF fields transient since object inspectors are not serializable. 6a4b671 [Michael Armbrust] Add option to avoid binding operators expressions automatically. 92cca08 [Michael Armbrust] Always include serialization debug info when running tests. 1279df2 [Michael Armbrust] Increase default number of partitions. --- project/SparkBuild.scala | 1 + .../catalyst/expressions/BoundAttribute.scala | 6 + .../sql/catalyst/expressions/Projection.scala | 6 +- .../sql/catalyst/expressions/aggregates.scala | 16 +- .../rdd/PartitionLocalRDDFunctions.scala | 100 ---------- .../apache/spark/sql/execution/Exchange.scala | 2 +- .../spark/sql/execution/aggregates.scala | 183 +++++++++++++----- .../org/apache/spark/sql/hive/hiveUdfs.scala | 3 + 8 files changed, 157 insertions(+), 160 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d1e4b8b964b88..6b8740d9f21a1 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -178,6 +178,7 @@ object SparkBuild extends Build { fork := true, javaOptions in Test += "-Dspark.home=" + sparkHome, javaOptions in Test += "-Dspark.testing=1", + javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark").map { case (k,v) => s"-D$k=$v" }.toSeq, javaOptions += "-Xmx3g", // Show full stack trace and duration in test cases. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index f70e80b7f27f2..37b9035df9d8c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -48,11 +48,17 @@ case class BoundReference(ordinal: Int, baseReference: Attribute) override def apply(input: Row): Any = input(ordinal) } +/** + * Used to denote operators that do their own binding of attributes internally. + */ +trait NoBind { self: trees.TreeNode[_] => } + class BindReferences[TreeNode <: QueryPlan[TreeNode]] extends Rule[TreeNode] { import BindReferences._ def apply(plan: TreeNode): TreeNode = { plan.transform { + case n: NoBind => n.asInstanceOf[TreeNode] case leafNode if leafNode.children.isEmpty => leafNode case unaryNode if unaryNode.children.size == 1 => unaryNode.transformExpressions { case e => bindReference(e, unaryNode.children.head.output) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 38542d3fc7290..5576ecbb65ef5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -28,9 +28,9 @@ class Projection(expressions: Seq[Expression]) extends (Row => Row) { protected val exprArray = expressions.toArray def apply(input: Row): Row = { - val outputArray = new Array[Any](exprArray.size) + val outputArray = new Array[Any](exprArray.length) var i = 0 - while (i < exprArray.size) { + while (i < exprArray.length) { outputArray(i) = exprArray(i).apply(input) i += 1 } @@ -57,7 +57,7 @@ case class MutableProjection(expressions: Seq[Expression]) extends (Row => Row) def apply(input: Row): Row = { var i = 0 - while (i < exprArray.size) { + while (i < exprArray.length) { mutableRow(i) = exprArray(i).apply(input) i += 1 } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 7303b155cae3d..53b884a41e16b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -27,7 +27,7 @@ abstract class AggregateExpression extends Expression { * Creates a new instance that can be used to compute this aggregate expression for a group * of input rows/ */ - def newInstance: AggregateFunction + def newInstance(): AggregateFunction } /** @@ -75,7 +75,7 @@ abstract class AggregateFunction override def apply(input: Row): Any // Do we really need this? - def newInstance = makeCopy(productIterator.map { case a: AnyRef => a }.toArray) + def newInstance() = makeCopy(productIterator.map { case a: AnyRef => a }.toArray) } case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -89,7 +89,7 @@ case class Count(child: Expression) extends PartialAggregate with trees.UnaryNod SplitEvaluation(Sum(partialCount.toAttribute), partialCount :: Nil) } - override def newInstance = new CountFunction(child, this) + override def newInstance()= new CountFunction(child, this) } case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpression { @@ -98,7 +98,7 @@ case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpressi def nullable = false def dataType = IntegerType override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")}})" - override def newInstance = new CountDistinctFunction(expressions, this) + override def newInstance()= new CountDistinctFunction(expressions, this) } case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -118,7 +118,7 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN partialCount :: partialSum :: Nil) } - override def newInstance = new AverageFunction(child, this) + override def newInstance()= new AverageFunction(child, this) } case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -134,7 +134,7 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ partialSum :: Nil) } - override def newInstance = new SumFunction(child, this) + override def newInstance()= new SumFunction(child, this) } case class SumDistinct(child: Expression) @@ -145,7 +145,7 @@ case class SumDistinct(child: Expression) def dataType = child.dataType override def toString = s"SUM(DISTINCT $child)" - override def newInstance = new SumDistinctFunction(child, this) + override def newInstance()= new SumDistinctFunction(child, this) } case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -160,7 +160,7 @@ case class First(child: Expression) extends PartialAggregate with trees.UnaryNod First(partialFirst.toAttribute), partialFirst :: Nil) } - override def newInstance = new FirstFunction(child, this) + override def newInstance()= new FirstFunction(child, this) } case class AverageFunction(expr: Expression, base: AggregateExpression) diff --git a/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala b/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala deleted file mode 100644 index f1230e7526ab1..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala +++ /dev/null @@ -1,100 +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.rdd - -import scala.language.implicitConversions - -import scala.reflect._ -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.{Aggregator, InterruptibleIterator, Logging} -import org.apache.spark.util.collection.AppendOnlyMap - -/* Implicit conversions */ -import org.apache.spark.SparkContext._ - -/** - * Extra functions on RDDs that perform only local operations. These can be used when data has - * already been partitioned correctly. - */ -private[spark] class PartitionLocalRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) - extends Logging - with Serializable { - - /** - * Cogroup corresponding partitions of `this` and `other`. These two RDDs should have - * the same number of partitions. Partitions of these two RDDs are cogrouped - * according to the indexes of partitions. If we have two RDDs and - * each of them has n partitions, we will cogroup the partition i from `this` - * with the partition i from `other`. - * This function will not introduce a shuffling operation. - */ - def cogroupLocally[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { - val cg = self.zipPartitions(other)((iter1:Iterator[(K, V)], iter2:Iterator[(K, W)]) => { - val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]] - - val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => { - if (hadVal) oldVal else Array.fill(2)(new ArrayBuffer[Any]) - } - - val getSeq = (k: K) => { - map.changeValue(k, update) - } - - iter1.foreach { kv => getSeq(kv._1)(0) += kv._2 } - iter2.foreach { kv => getSeq(kv._1)(1) += kv._2 } - - map.iterator - }).mapValues { case Seq(vs, ws) => (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])} - - cg - } - - /** - * Group the values for each key within a partition of the RDD into a single sequence. - * This function will not introduce a shuffling operation. - */ - def groupByKeyLocally(): RDD[(K, Seq[V])] = { - def createCombiner(v: V) = ArrayBuffer(v) - def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v - val aggregator = new Aggregator[K, V, ArrayBuffer[V]](createCombiner, mergeValue, _ ++ _) - val bufs = self.mapPartitionsWithContext((context, iter) => { - new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context)) - }, preservesPartitioning = true) - bufs.asInstanceOf[RDD[(K, Seq[V])]] - } - - /** - * Join corresponding partitions of `this` and `other`. - * If we have two RDDs and each of them has n partitions, - * we will join the partition i from `this` with the partition i from `other`. - * This function will not introduce a shuffling operation. - */ - def joinLocally[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = { - cogroupLocally(other).flatMapValues { - case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (v, w) - } - } -} - -private[spark] object PartitionLocalRDDFunctions { - implicit def rddToPartitionLocalRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = - new PartitionLocalRDDFunctions(rdd) -} - - diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 869673b1fe978..450c142c0baa4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -76,7 +76,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una */ object AddExchange extends Rule[SparkPlan] { // TODO: Determine the number of partitions. - val numPartitions = 8 + val numPartitions = 150 def apply(plan: SparkPlan): SparkPlan = plan.transformUp { case operator: SparkPlan => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala index 8515a18f18c55..2a4f7b5670457 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala @@ -17,14 +17,13 @@ package org.apache.spark.sql.execution +import java.util.HashMap + import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ -/* Implicit conversions */ -import org.apache.spark.rdd.PartitionLocalRDDFunctions._ - /** * Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each * group. @@ -40,7 +39,7 @@ case class Aggregate( groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], child: SparkPlan)(@transient sc: SparkContext) - extends UnaryNode { + extends UnaryNode with NoBind { override def requiredChildDistribution = if (partial) { @@ -55,61 +54,149 @@ case class Aggregate( override def otherCopyArgs = sc :: Nil + // HACK: Generators don't correctly preserve their output through serializations so we grab + // out child's output attributes statically here. + val childOutput = child.output + def output = aggregateExpressions.map(_.toAttribute) - /* Replace all aggregate expressions with spark functions that will compute the result. */ - def createAggregateImplementations() = aggregateExpressions.map { agg => - val impl = agg transform { - case a: AggregateExpression => a.newInstance + /** + * An aggregate that needs to be computed for each row in a group. + * + * @param unbound Unbound version of this aggregate, used for result substitution. + * @param aggregate A bound copy of this aggregate used to create a new aggregation buffer. + * @param resultAttribute An attribute used to refer to the result of this aggregate in the final + * output. + */ + case class ComputedAggregate( + unbound: AggregateExpression, + aggregate: AggregateExpression, + resultAttribute: AttributeReference) + + /** A list of aggregates that need to be computed for each group. */ + @transient + lazy val computedAggregates = aggregateExpressions.flatMap { agg => + agg.collect { + case a: AggregateExpression => + ComputedAggregate( + a, + BindReferences.bindReference(a, childOutput).asInstanceOf[AggregateExpression], + AttributeReference(s"aggResult:$a", a.dataType, nullable = true)()) } + }.toArray + + /** The schema of the result of all aggregate evaluations */ + @transient + lazy val computedSchema = computedAggregates.map(_.resultAttribute) + + /** Creates a new aggregate buffer for a group. */ + def newAggregateBuffer(): Array[AggregateFunction] = { + val buffer = new Array[AggregateFunction](computedAggregates.length) + var i = 0 + while (i < computedAggregates.length) { + buffer(i) = computedAggregates(i).aggregate.newInstance() + i += 1 + } + buffer + } - val remainingAttributes = impl.collect { case a: Attribute => a } - // If any references exist that are not inside agg functions then the must be grouping exprs - // in this case we must rebind them to the grouping tuple. - if (remainingAttributes.nonEmpty) { - val unaliasedAggregateExpr = agg transform { case Alias(c, _) => c } - - // An exact match with a grouping expression - val exactGroupingExpr = groupingExpressions.indexOf(unaliasedAggregateExpr) match { - case -1 => None - case ordinal => Some(BoundReference(ordinal, Alias(impl, "AGGEXPR")().toAttribute)) - } + /** Named attributes used to substitute grouping attributes into the final result. */ + @transient + lazy val namedGroups = groupingExpressions.map { + case ne: NamedExpression => ne -> ne.toAttribute + case e => e -> Alias(e, s"groupingExpr:$e")().toAttribute + } - exactGroupingExpr.getOrElse( - sys.error(s"$agg is not in grouping expressions: $groupingExpressions")) - } else { - impl + /** + * A map of substitutions that are used to insert the aggregate expressions and grouping + * expression into the final result expression. + */ + @transient + lazy val resultMap = + (computedAggregates.map { agg => agg.unbound -> agg.resultAttribute} ++ namedGroups).toMap + + /** + * Substituted version of aggregateExpressions expressions which are used to compute final + * output rows given a group and the result of all aggregate computations. + */ + @transient + lazy val resultExpressions = aggregateExpressions.map { agg => + agg.transform { + case e: Expression if resultMap.contains(e) => resultMap(e) } } def execute() = attachTree(this, "execute") { - // TODO: If the child of it is an [[catalyst.execution.Exchange]], - // do not evaluate the groupingExpressions again since we have evaluated it - // in the [[catalyst.execution.Exchange]]. - val grouped = child.execute().mapPartitions { iter => - val buildGrouping = new Projection(groupingExpressions) - iter.map(row => (buildGrouping(row), row.copy())) - }.groupByKeyLocally() - - val result = grouped.map { case (group, rows) => - val aggImplementations = createAggregateImplementations() - - // Pull out all the functions so we can feed each row into them. - val aggFunctions = aggImplementations.flatMap(_ collect { case f: AggregateFunction => f }) - - rows.foreach { row => - aggFunctions.foreach(_.update(row)) + if (groupingExpressions.isEmpty) { + child.execute().mapPartitions { iter => + val buffer = newAggregateBuffer() + var currentRow: Row = null + while (iter.hasNext) { + currentRow = iter.next() + var i = 0 + while (i < buffer.length) { + buffer(i).update(currentRow) + i += 1 + } + } + val resultProjection = new Projection(resultExpressions, computedSchema) + val aggregateResults = new GenericMutableRow(computedAggregates.length) + + var i = 0 + while (i < buffer.length) { + aggregateResults(i) = buffer(i).apply(EmptyRow) + i += 1 + } + + Iterator(resultProjection(aggregateResults)) } - buildRow(aggImplementations.map(_.apply(group))) - } - - // TODO: THIS BREAKS PIPELINING, DOUBLE COMPUTES THE ANSWER, AND USES TOO MUCH MEMORY... - if (groupingExpressions.isEmpty && result.count == 0) { - // When there there is no output to the Aggregate operator, we still output an empty row. - val aggImplementations = createAggregateImplementations() - sc.makeRDD(buildRow(aggImplementations.map(_.apply(null))) :: Nil) } else { - result + child.execute().mapPartitions { iter => + val hashTable = new HashMap[Row, Array[AggregateFunction]] + val groupingProjection = new MutableProjection(groupingExpressions, childOutput) + + var currentRow: Row = null + while (iter.hasNext) { + currentRow = iter.next() + val currentGroup = groupingProjection(currentRow) + var currentBuffer = hashTable.get(currentGroup) + if (currentBuffer == null) { + currentBuffer = newAggregateBuffer() + hashTable.put(currentGroup.copy(), currentBuffer) + } + + var i = 0 + while (i < currentBuffer.length) { + currentBuffer(i).update(currentRow) + i += 1 + } + } + + new Iterator[Row] { + private[this] val hashTableIter = hashTable.entrySet().iterator() + private[this] val aggregateResults = new GenericMutableRow(computedAggregates.length) + private[this] val resultProjection = + new MutableProjection(resultExpressions, computedSchema ++ namedGroups.map(_._2)) + private[this] val joinedRow = new JoinedRow + + override final def hasNext: Boolean = hashTableIter.hasNext + + override final def next(): Row = { + val currentEntry = hashTableIter.next() + val currentGroup = currentEntry.getKey + val currentBuffer = currentEntry.getValue + + var i = 0 + while (i < currentBuffer.length) { + // Evaluating an aggregate buffer returns the result. No row is required since we + // already added all rows in the group using update. + aggregateResults(i) = currentBuffer(i).apply(EmptyRow) + i += 1 + } + resultProjection(joinedRow(aggregateResults, currentGroup)) + } + } + } } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 44901db3f963b..2c607455c8e4b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -337,13 +337,16 @@ case class HiveGenericUdaf( type UDFType = AbstractGenericUDAFResolver + @transient protected lazy val resolver: AbstractGenericUDAFResolver = createFunction(name) + @transient protected lazy val objectInspector = { resolver.getEvaluator(children.map(_.dataType.toTypeInfo).toArray) .init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors.toArray) } + @transient protected lazy val inspectors = children.map(_.dataType).map(toInspector) def dataType: DataType = inspectorToDataType(objectInspector) From b5bae849dbdf833a80746b705732b8c86971fddc Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 7 Apr 2014 01:46:50 -0700 Subject: [PATCH 013/641] [SQL] SPARK-1427 Fix toString for SchemaRDD NativeCommands. Author: Michael Armbrust Closes #343 from marmbrus/toStringFix and squashes the following commits: 37198fe [Michael Armbrust] Fix toString for SchemaRDD NativeCommands. --- .../src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 ++ .../src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala | 3 +-- .../main/scala/org/apache/spark/sql/hive/HiveContext.scala | 6 ++++++ .../apache/spark/sql/hive/execution/HiveQuerySuite.scala | 4 ++++ 4 files changed, 13 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 36059c6630aa4..3193787680d16 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -224,6 +224,8 @@ class SQLContext(@transient val sparkContext: SparkContext) protected def stringOrError[A](f: => A): String = try f.toString catch { case e: Throwable => e.toString } + def simpleString: String = stringOrError(executedPlan) + override def toString: String = s"""== Logical Plan == |${stringOrError(analyzed)} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 840803a52c1cf..3dd9897c0d3b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -41,8 +41,7 @@ trait SchemaRDDLike { override def toString = s"""${super.toString} |== Query Plan == - |${queryExecution.executedPlan}""".stripMargin.trim - + |${queryExecution.simpleString}""".stripMargin.trim /** * Saves the contents of this `SchemaRDD` as a parquet file, preserving the schema. Files that diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index f66a667c0a942..353458432b210 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -297,5 +297,11 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { val asString = result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")).toSeq asString } + + override def simpleString: String = + logical match { + case _: NativeCommand => "" + case _ => executedPlan.toString + } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 0c27498a93507..a09667ac84b01 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -146,4 +146,8 @@ class HiveQuerySuite extends HiveComparisonTest { hql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s") } + test("SchemaRDD toString") { + hql("SHOW TABLES").toString + hql("SELECT * FROM src").toString + } } From a3c51c6ea2320efdeb2a6a5c1cd11d714f8994aa Mon Sep 17 00:00:00 2001 From: Davis Shepherd Date: Mon, 7 Apr 2014 10:02:00 -0700 Subject: [PATCH 014/641] SPARK-1432: Make sure that all metadata fields are properly cleaned While working on spark-1337 with @pwendell, we noticed that not all of the metadata maps in JobProgessListener were being properly cleaned. This could lead to a (hypothetical) memory leak issue should a job run long enough. This patch aims to address the issue. Author: Davis Shepherd Closes #338 from dgshep/master and squashes the following commits: a77b65c [Davis Shepherd] In the contex of SPARK-1337: Make sure that all metadata fields are properly cleaned --- .../scala/org/apache/spark/ui/jobs/JobProgressListener.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index cd4be57227a16..048f671c8788f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -83,7 +83,6 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { if (stages.size > retainedStages) { val toRemove = math.max(retainedStages / 10, 1) stages.take(toRemove).foreach { s => - stageIdToTaskData.remove(s.stageId) stageIdToTime.remove(s.stageId) stageIdToShuffleRead.remove(s.stageId) stageIdToShuffleWrite.remove(s.stageId) @@ -92,8 +91,10 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { stageIdToTasksActive.remove(s.stageId) stageIdToTasksComplete.remove(s.stageId) stageIdToTasksFailed.remove(s.stageId) + stageIdToTaskData.remove(s.stageId) + stageIdToExecutorSummaries.remove(s.stageId) stageIdToPool.remove(s.stageId) - if (stageIdToDescription.contains(s.stageId)) {stageIdToDescription.remove(s.stageId)} + stageIdToDescription.remove(s.stageId) } stages.trimStart(toRemove) } From 83f2a2f14e4145a04672e42216d43100a66b1fc2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 7 Apr 2014 10:45:31 -0700 Subject: [PATCH 015/641] [sql] Rename Expression.apply to eval for better readability. Also used this opportunity to add a bunch of override's and made some members private. Author: Reynold Xin Closes #340 from rxin/eval and squashes the following commits: a7c7ca7 [Reynold Xin] Fixed conflicts in merge. 9069de6 [Reynold Xin] Merge branch 'master' into eval 3ccc313 [Reynold Xin] Merge branch 'master' into eval 1a47e10 [Reynold Xin] Renamed apply to eval for generators and added a bunch of override's. ea061de [Reynold Xin] Rename Expression.apply to eval for better readability. --- .../catalyst/expressions/BoundAttribute.scala | 2 +- .../spark/sql/catalyst/expressions/Cast.scala | 4 +- .../sql/catalyst/expressions/Expression.scala | 26 ++--- .../sql/catalyst/expressions/Projection.scala | 5 +- .../spark/sql/catalyst/expressions/Row.scala | 4 +- .../sql/catalyst/expressions/ScalaUdf.scala | 8 +- .../catalyst/expressions/WrapDynamic.scala | 2 +- .../sql/catalyst/expressions/aggregates.scala | 96 +++++++++---------- .../sql/catalyst/expressions/arithmetic.scala | 12 +-- .../catalyst/expressions/complexTypes.scala | 14 +-- .../sql/catalyst/expressions/generators.scala | 20 ++-- .../sql/catalyst/expressions/literals.scala | 6 +- .../expressions/namedExpressions.scala | 2 +- .../catalyst/expressions/nullFunctions.scala | 12 +-- .../sql/catalyst/expressions/predicates.scala | 46 ++++----- .../expressions/stringOperations.scala | 10 +- .../sql/catalyst/optimizer/Optimizer.scala | 2 +- .../ExpressionEvaluationSuite.scala | 8 +- .../apache/spark/sql/execution/Generate.scala | 8 +- .../spark/sql/execution/aggregates.scala | 4 +- .../spark/sql/execution/basicOperators.scala | 2 +- .../apache/spark/sql/execution/TgfSuite.scala | 6 +- .../apache/spark/sql/hive/hiveOperators.scala | 4 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 12 +-- 24 files changed, 156 insertions(+), 159 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 37b9035df9d8c..4ebf6c4584b94 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -45,7 +45,7 @@ case class BoundReference(ordinal: Int, baseReference: Attribute) override def toString = s"$baseReference:$ordinal" - override def apply(input: Row): Any = input(ordinal) + override def eval(input: Row): Any = input(ordinal) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 941b53fe70d23..89226999ca005 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -185,8 +185,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case DoubleType => castToDouble } - override def apply(input: Row): Any = { - val evaluated = child.apply(input) + override def eval(input: Row): Any = { + val evaluated = child.eval(input) if (evaluated == null) { null } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index a3d19525503ba..f190bd0cca375 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.types.{DataType, FractionalType, IntegralType, NumericType, NativeType} @@ -50,7 +50,7 @@ abstract class Expression extends TreeNode[Expression] { def references: Set[Attribute] /** Returns the result of evaluating this expression on a given input Row */ - def apply(input: Row = null): EvaluatedType = + def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") /** @@ -73,7 +73,7 @@ abstract class Expression extends TreeNode[Expression] { */ @inline def n1(e: Expression, i: Row, f: ((Numeric[Any], Any) => Any)): Any = { - val evalE = e.apply(i) + val evalE = e.eval(i) if (evalE == null) { null } else { @@ -102,11 +102,11 @@ abstract class Expression extends TreeNode[Expression] { throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") } - val evalE1 = e1.apply(i) + val evalE1 = e1.eval(i) if(evalE1 == null) { null } else { - val evalE2 = e2.apply(i) + val evalE2 = e2.eval(i) if (evalE2 == null) { null } else { @@ -135,11 +135,11 @@ abstract class Expression extends TreeNode[Expression] { throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") } - val evalE1 = e1.apply(i: Row) + val evalE1 = e1.eval(i: Row) if(evalE1 == null) { null } else { - val evalE2 = e2.apply(i: Row) + val evalE2 = e2.eval(i: Row) if (evalE2 == null) { null } else { @@ -168,11 +168,11 @@ abstract class Expression extends TreeNode[Expression] { throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") } - val evalE1 = e1.apply(i) + val evalE1 = e1.eval(i) if(evalE1 == null) { null } else { - val evalE2 = e2.apply(i) + val evalE2 = e2.eval(i) if (evalE2 == null) { null } else { @@ -205,11 +205,11 @@ abstract class Expression extends TreeNode[Expression] { throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") } - val evalE1 = e1.apply(i) + val evalE1 = e1.eval(i) if(evalE1 == null) { null } else { - val evalE2 = e2.apply(i) + val evalE2 = e2.eval(i) if (evalE2 == null) { null } else { @@ -231,7 +231,7 @@ abstract class BinaryExpression extends Expression with trees.BinaryNode[Express override def foldable = left.foldable && right.foldable - def references = left.references ++ right.references + override def references = left.references ++ right.references override def toString = s"($left $symbol $right)" } @@ -243,5 +243,5 @@ abstract class LeafExpression extends Expression with trees.LeafNode[Expression] abstract class UnaryExpression extends Expression with trees.UnaryNode[Expression] { self: Product => - def references = child.references + override def references = child.references } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 5576ecbb65ef5..c9b7cea6a3e5f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -27,11 +27,12 @@ class Projection(expressions: Seq[Expression]) extends (Row => Row) { this(expressions.map(BindReferences.bindReference(_, inputSchema))) protected val exprArray = expressions.toArray + def apply(input: Row): Row = { val outputArray = new Array[Any](exprArray.length) var i = 0 while (i < exprArray.length) { - outputArray(i) = exprArray(i).apply(input) + outputArray(i) = exprArray(i).eval(input) i += 1 } new GenericRow(outputArray) @@ -58,7 +59,7 @@ case class MutableProjection(expressions: Seq[Expression]) extends (Row => Row) def apply(input: Row): Row = { var i = 0 while (i < exprArray.length) { - mutableRow(i) = exprArray(i).apply(input) + mutableRow(i) = exprArray(i).eval(input) i += 1 } mutableRow diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 9f4d84466e6ac..0f06ea088e1a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -212,8 +212,8 @@ class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { var i = 0 while (i < ordering.size) { val order = ordering(i) - val left = order.child.apply(a) - val right = order.child.apply(b) + val left = order.child.eval(a) + val right = order.child.eval(b) if (left == null && right == null) { // Both null, continue looking. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index f53d8504b083f..5e089f7618e0a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -27,13 +27,13 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi def references = children.flatMap(_.references).toSet def nullable = true - override def apply(input: Row): Any = { + override def eval(input: Row): Any = { children.size match { - case 1 => function.asInstanceOf[(Any) => Any](children(0).apply(input)) + case 1 => function.asInstanceOf[(Any) => Any](children(0).eval(input)) case 2 => function.asInstanceOf[(Any, Any) => Any]( - children(0).apply(input), - children(1).apply(input)) + children(0).eval(input), + children(1).eval(input)) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala index 9828d0b9bd8b2..e787c59e75723 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala @@ -30,7 +30,7 @@ case class WrapDynamic(children: Seq[Attribute]) extends Expression { def references = children.toSet def dataType = DynamicType - override def apply(input: Row): DynamicRow = input match { + override def eval(input: Row): DynamicRow = input match { // Avoid copy for generic rows. case g: GenericRow => new DynamicRow(children, g.values) case otherRowType => new DynamicRow(children, otherRowType.toArray) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 53b884a41e16b..5edcea14278c7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -43,7 +43,7 @@ case class SplitEvaluation( partialEvaluations: Seq[NamedExpression]) /** - * An [[AggregateExpression]] that can be partially computed without seeing all relevent tuples. + * An [[AggregateExpression]] that can be partially computed without seeing all relevant tuples. * These partial evaluations can then be combined to compute the actual answer. */ abstract class PartialAggregate extends AggregateExpression { @@ -63,28 +63,28 @@ abstract class AggregateFunction extends AggregateExpression with Serializable with trees.LeafNode[Expression] { self: Product => - type EvaluatedType = Any + override type EvaluatedType = Any /** Base should return the generic aggregate expression that this function is computing */ val base: AggregateExpression - def references = base.references - def nullable = base.nullable - def dataType = base.dataType + override def references = base.references + override def nullable = base.nullable + override def dataType = base.dataType def update(input: Row): Unit - override def apply(input: Row): Any + override def eval(input: Row): Any // Do we really need this? - def newInstance() = makeCopy(productIterator.map { case a: AnyRef => a }.toArray) + override def newInstance() = makeCopy(productIterator.map { case a: AnyRef => a }.toArray) } case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - def references = child.references - def nullable = false - def dataType = IntegerType + override def references = child.references + override def nullable = false + override def dataType = IntegerType override def toString = s"COUNT($child)" - def asPartial: SplitEvaluation = { + override def asPartial: SplitEvaluation = { val partialCount = Alias(Count(child), "PartialCount")() SplitEvaluation(Sum(partialCount.toAttribute), partialCount :: Nil) } @@ -93,18 +93,18 @@ case class Count(child: Expression) extends PartialAggregate with trees.UnaryNod } case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpression { - def children = expressions - def references = expressions.flatMap(_.references).toSet - def nullable = false - def dataType = IntegerType + override def children = expressions + override def references = expressions.flatMap(_.references).toSet + override def nullable = false + override def dataType = IntegerType override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")}})" override def newInstance()= new CountDistinctFunction(expressions, this) } case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - def references = child.references - def nullable = false - def dataType = DoubleType + override def references = child.references + override def nullable = false + override def dataType = DoubleType override def toString = s"AVG($child)" override def asPartial: SplitEvaluation = { @@ -122,9 +122,9 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN } case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - def references = child.references - def nullable = false - def dataType = child.dataType + override def references = child.references + override def nullable = false + override def dataType = child.dataType override def toString = s"SUM($child)" override def asPartial: SplitEvaluation = { @@ -140,18 +140,18 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ case class SumDistinct(child: Expression) extends AggregateExpression with trees.UnaryNode[Expression] { - def references = child.references - def nullable = false - def dataType = child.dataType + override def references = child.references + override def nullable = false + override def dataType = child.dataType override def toString = s"SUM(DISTINCT $child)" override def newInstance()= new SumDistinctFunction(child, this) } case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - def references = child.references - def nullable = child.nullable - def dataType = child.dataType + override def references = child.references + override def nullable = child.nullable + override def dataType = child.dataType override def toString = s"FIRST($child)" override def asPartial: SplitEvaluation = { @@ -169,17 +169,15 @@ case class AverageFunction(expr: Expression, base: AggregateExpression) def this() = this(null, null) // Required for serialization. private var count: Long = _ - private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).apply(EmptyRow)) + private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).eval(EmptyRow)) private val sumAsDouble = Cast(sum, DoubleType) - - private val addFunction = Add(sum, expr) - override def apply(input: Row): Any = - sumAsDouble.apply(EmptyRow).asInstanceOf[Double] / count.toDouble + override def eval(input: Row): Any = + sumAsDouble.eval(EmptyRow).asInstanceOf[Double] / count.toDouble - def update(input: Row): Unit = { + override def update(input: Row): Unit = { count += 1 sum.update(addFunction, input) } @@ -190,28 +188,28 @@ case class CountFunction(expr: Expression, base: AggregateExpression) extends Ag var count: Int = _ - def update(input: Row): Unit = { - val evaluatedExpr = expr.map(_.apply(input)) + override def update(input: Row): Unit = { + val evaluatedExpr = expr.map(_.eval(input)) if (evaluatedExpr.map(_ != null).reduceLeft(_ || _)) { count += 1 } } - override def apply(input: Row): Any = count + override def eval(input: Row): Any = count } case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. - private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).apply(null)) + private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).eval(null)) private val addFunction = Add(sum, expr) - def update(input: Row): Unit = { + override def update(input: Row): Unit = { sum.update(addFunction, input) } - override def apply(input: Row): Any = sum.apply(null) + override def eval(input: Row): Any = sum.eval(null) } case class SumDistinctFunction(expr: Expression, base: AggregateExpression) @@ -219,16 +217,16 @@ case class SumDistinctFunction(expr: Expression, base: AggregateExpression) def this() = this(null, null) // Required for serialization. - val seen = new scala.collection.mutable.HashSet[Any]() + private val seen = new scala.collection.mutable.HashSet[Any]() - def update(input: Row): Unit = { - val evaluatedExpr = expr.apply(input) + override def update(input: Row): Unit = { + val evaluatedExpr = expr.eval(input) if (evaluatedExpr != null) { seen += evaluatedExpr } } - override def apply(input: Row): Any = + override def eval(input: Row): Any = seen.reduceLeft(base.dataType.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].plus) } @@ -239,14 +237,14 @@ case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpressio val seen = new scala.collection.mutable.HashSet[Any]() - def update(input: Row): Unit = { - val evaluatedExpr = expr.map(_.apply(input)) + override def update(input: Row): Unit = { + val evaluatedExpr = expr.map(_.eval(input)) if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) { seen += evaluatedExpr } } - override def apply(input: Row): Any = seen.size + override def eval(input: Row): Any = seen.size } case class FirstFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -254,11 +252,11 @@ case class FirstFunction(expr: Expression, base: AggregateExpression) extends Ag var result: Any = null - def update(input: Row): Unit = { + override def update(input: Row): Unit = { if (result == null) { - result = expr.apply(input) + result = expr.eval(input) } } - override def apply(input: Row): Any = result + override def eval(input: Row): Any = result } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index fba056e7c07e3..c79c1847cedf5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -28,7 +28,7 @@ case class UnaryMinus(child: Expression) extends UnaryExpression { def nullable = child.nullable override def toString = s"-$child" - override def apply(input: Row): Any = { + override def eval(input: Row): Any = { n1(child, input, _.negate(_)) } } @@ -55,25 +55,25 @@ abstract class BinaryArithmetic extends BinaryExpression { case class Add(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "+" - override def apply(input: Row): Any = n2(input, left, right, _.plus(_, _)) + override def eval(input: Row): Any = n2(input, left, right, _.plus(_, _)) } case class Subtract(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "-" - override def apply(input: Row): Any = n2(input, left, right, _.minus(_, _)) + override def eval(input: Row): Any = n2(input, left, right, _.minus(_, _)) } case class Multiply(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "*" - override def apply(input: Row): Any = n2(input, left, right, _.times(_, _)) + override def eval(input: Row): Any = n2(input, left, right, _.times(_, _)) } case class Divide(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "/" - override def apply(input: Row): Any = dataType match { + override def eval(input: Row): Any = dataType match { case _: FractionalType => f2(input, left, right, _.div(_, _)) case _: IntegralType => i2(input, left , right, _.quot(_, _)) } @@ -83,5 +83,5 @@ case class Divide(left: Expression, right: Expression) extends BinaryArithmetic case class Remainder(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "%" - override def apply(input: Row): Any = i2(input, left, right, _.rem(_, _)) + override def eval(input: Row): Any = i2(input, left, right, _.rem(_, _)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index ab96618d73df7..c947155cb701c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -39,10 +39,10 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { override def toString = s"$child[$ordinal]" - override def apply(input: Row): Any = { + override def eval(input: Row): Any = { if (child.dataType.isInstanceOf[ArrayType]) { - val baseValue = child.apply(input).asInstanceOf[Seq[_]] - val o = ordinal.apply(input).asInstanceOf[Int] + val baseValue = child.eval(input).asInstanceOf[Seq[_]] + val o = ordinal.eval(input).asInstanceOf[Int] if (baseValue == null) { null } else if (o >= baseValue.size || o < 0) { @@ -51,8 +51,8 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { baseValue(o) } } else { - val baseValue = child.apply(input).asInstanceOf[Map[Any, _]] - val key = ordinal.apply(input) + val baseValue = child.eval(input).asInstanceOf[Map[Any, _]] + val key = ordinal.eval(input) if (baseValue == null) { null } else { @@ -85,8 +85,8 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio override lazy val resolved = childrenResolved && child.dataType.isInstanceOf[StructType] - override def apply(input: Row): Any = { - val baseValue = child.apply(input).asInstanceOf[Row] + override def eval(input: Row): Any = { + val baseValue = child.eval(input).asInstanceOf[Row] if (baseValue == null) null else baseValue(ordinal) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index e9b491b10a5f2..dd78614754e12 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -35,17 +35,17 @@ import org.apache.spark.sql.catalyst.types._ * requested. The attributes produced by this function will be automatically copied anytime rules * result in changes to the Generator or its children. */ -abstract class Generator extends Expression with (Row => TraversableOnce[Row]) { +abstract class Generator extends Expression { self: Product => - type EvaluatedType = TraversableOnce[Row] + override type EvaluatedType = TraversableOnce[Row] - lazy val dataType = + override lazy val dataType = ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable)))) - def nullable = false + override def nullable = false - def references = children.flatMap(_.references).toSet + override def references = children.flatMap(_.references).toSet /** * Should be overridden by specific generators. Called only once for each instance to ensure @@ -63,7 +63,7 @@ abstract class Generator extends Expression with (Row => TraversableOnce[Row]) { } /** Should be implemented by child classes to perform specific Generators. */ - def apply(input: Row): TraversableOnce[Row] + override def eval(input: Row): TraversableOnce[Row] /** Overridden `makeCopy` also copies the attributes that are produced by this generator. */ override def makeCopy(newArgs: Array[AnyRef]): this.type = { @@ -83,7 +83,7 @@ case class Explode(attributeNames: Seq[String], child: Expression) child.resolved && (child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType]) - lazy val elementTypes = child.dataType match { + private lazy val elementTypes = child.dataType match { case ArrayType(et) => et :: Nil case MapType(kt,vt) => kt :: vt :: Nil } @@ -100,13 +100,13 @@ case class Explode(attributeNames: Seq[String], child: Expression) } } - override def apply(input: Row): TraversableOnce[Row] = { + override def eval(input: Row): TraversableOnce[Row] = { child.dataType match { case ArrayType(_) => - val inputArray = child.apply(input).asInstanceOf[Seq[Any]] + val inputArray = child.eval(input).asInstanceOf[Seq[Any]] if (inputArray == null) Nil else inputArray.map(v => new GenericRow(Array(v))) case MapType(_, _) => - val inputMap = child.apply(input).asInstanceOf[Map[Any,Any]] + val inputMap = child.eval(input).asInstanceOf[Map[Any,Any]] if (inputMap == null) Nil else inputMap.map { case (k,v) => new GenericRow(Array(k,v)) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index d879b2b5e8ba1..e15e16d633365 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -57,7 +57,7 @@ case class Literal(value: Any, dataType: DataType) extends LeafExpression { override def toString = if (value != null) value.toString else "null" type EvaluatedType = Any - override def apply(input: Row):Any = value + override def eval(input: Row):Any = value } // TODO: Specialize @@ -69,8 +69,8 @@ case class MutableLiteral(var value: Any, nullable: Boolean = true) extends Leaf def references = Set.empty def update(expression: Expression, input: Row) = { - value = expression.apply(input) + value = expression.eval(input) } - override def apply(input: Row) = value + override def eval(input: Row) = value } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 69c8bed309c18..eb4bc8e755284 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -79,7 +79,7 @@ case class Alias(child: Expression, name: String) type EvaluatedType = Any - override def apply(input: Row) = child.apply(input) + override def eval(input: Row) = child.eval(input) def dataType = child.dataType def nullable = child.nullable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index 5a47768dcb4a1..ce6d99c911ab3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -41,11 +41,11 @@ case class Coalesce(children: Seq[Expression]) extends Expression { throw new UnresolvedException(this, "Coalesce cannot have children of different types.") } - override def apply(input: Row): Any = { + override def eval(input: Row): Any = { var i = 0 var result: Any = null while(i < children.size && result == null) { - result = children(i).apply(input) + result = children(i).eval(input) i += 1 } result @@ -57,8 +57,8 @@ case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expr override def foldable = child.foldable def nullable = false - override def apply(input: Row): Any = { - child.apply(input) == null + override def eval(input: Row): Any = { + child.eval(input) == null } } @@ -68,7 +68,7 @@ case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[E def nullable = false override def toString = s"IS NOT NULL $child" - override def apply(input: Row): Any = { - child.apply(input) != null + override def eval(input: Row): Any = { + child.eval(input) != null } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index b74809e5ca67d..da5b2cf5b0362 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.types.{BooleanType, StringType, TimestampTy object InterpretedPredicate { def apply(expression: Expression): (Row => Boolean) = { - (r: Row) => expression.apply(r).asInstanceOf[Boolean] + (r: Row) => expression.eval(r).asInstanceOf[Boolean] } } @@ -54,8 +54,8 @@ case class Not(child: Expression) extends Predicate with trees.UnaryNode[Express def nullable = child.nullable override def toString = s"NOT $child" - override def apply(input: Row): Any = { - child.apply(input) match { + override def eval(input: Row): Any = { + child.eval(input) match { case null => null case b: Boolean => !b } @@ -71,18 +71,18 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate { def nullable = true // TODO: Figure out correct nullability semantics of IN. override def toString = s"$value IN ${list.mkString("(", ",", ")")}" - override def apply(input: Row): Any = { - val evaluatedValue = value.apply(input) - list.exists(e => e.apply(input) == evaluatedValue) + override def eval(input: Row): Any = { + val evaluatedValue = value.eval(input) + list.exists(e => e.eval(input) == evaluatedValue) } } case class And(left: Expression, right: Expression) extends BinaryPredicate { def symbol = "&&" - override def apply(input: Row): Any = { - val l = left.apply(input) - val r = right.apply(input) + override def eval(input: Row): Any = { + val l = left.eval(input) + val r = right.eval(input) if (l == false || r == false) { false } else if (l == null || r == null ) { @@ -96,9 +96,9 @@ case class And(left: Expression, right: Expression) extends BinaryPredicate { case class Or(left: Expression, right: Expression) extends BinaryPredicate { def symbol = "||" - override def apply(input: Row): Any = { - val l = left.apply(input) - val r = right.apply(input) + override def eval(input: Row): Any = { + val l = left.eval(input) + val r = right.eval(input) if (l == true || r == true) { true } else if (l == null || r == null) { @@ -115,31 +115,31 @@ abstract class BinaryComparison extends BinaryPredicate { case class Equals(left: Expression, right: Expression) extends BinaryComparison { def symbol = "=" - override def apply(input: Row): Any = { - val l = left.apply(input) - val r = right.apply(input) + override def eval(input: Row): Any = { + val l = left.eval(input) + val r = right.eval(input) if (l == null || r == null) null else l == r } } case class LessThan(left: Expression, right: Expression) extends BinaryComparison { def symbol = "<" - override def apply(input: Row): Any = c2(input, left, right, _.lt(_, _)) + override def eval(input: Row): Any = c2(input, left, right, _.lt(_, _)) } case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { def symbol = "<=" - override def apply(input: Row): Any = c2(input, left, right, _.lteq(_, _)) + override def eval(input: Row): Any = c2(input, left, right, _.lteq(_, _)) } case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison { def symbol = ">" - override def apply(input: Row): Any = c2(input, left, right, _.gt(_, _)) + override def eval(input: Row): Any = c2(input, left, right, _.gt(_, _)) } case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { def symbol = ">=" - override def apply(input: Row): Any = c2(input, left, right, _.gteq(_, _)) + override def eval(input: Row): Any = c2(input, left, right, _.gteq(_, _)) } case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) @@ -159,11 +159,11 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi } type EvaluatedType = Any - override def apply(input: Row): Any = { - if (predicate(input).asInstanceOf[Boolean]) { - trueValue.apply(input) + override def eval(input: Row): Any = { + if (predicate.eval(input).asInstanceOf[Boolean]) { + trueValue.eval(input) } else { - falseValue.apply(input) + falseValue.eval(input) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index 42b7a9b125b7a..a27c71db1b999 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -22,8 +22,6 @@ import java.util.regex.Pattern import org.apache.spark.sql.catalyst.types.DataType import org.apache.spark.sql.catalyst.types.StringType import org.apache.spark.sql.catalyst.types.BooleanType -import org.apache.spark.sql.catalyst.trees.TreeNode -import org.apache.spark.sql.catalyst.errors.`package`.TreeNodeException trait StringRegexExpression { @@ -52,12 +50,12 @@ trait StringRegexExpression { protected def pattern(str: String) = if(cache == null) compile(str) else cache - override def apply(input: Row): Any = { - val l = left.apply(input) - if(l == null) { + override def eval(input: Row): Any = { + val l = left.eval(input) + if (l == null) { null } else { - val r = right.apply(input) + val r = right.eval(input) if(r == null) { null } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 3dd6818029bcf..37b23ba58289c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -45,7 +45,7 @@ object ConstantFolding extends Rule[LogicalPlan] { case q: LogicalPlan => q transformExpressionsDown { // Skip redundant folding of literals. case l: Literal => l - case e if e.foldable => Literal(e.apply(null), e.dataType) + case e if e.foldable => Literal(e.eval(null), e.dataType) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 43876033d327b..92987405aa313 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ class ExpressionEvaluationSuite extends FunSuite { test("literals") { - assert((Literal(1) + Literal(1)).apply(null) === 2) + assert((Literal(1) + Literal(1)).eval(null) === 2) } /** @@ -62,7 +62,7 @@ class ExpressionEvaluationSuite extends FunSuite { notTrueTable.foreach { case (v, answer) => val expr = Not(Literal(v, BooleanType)) - val result = expr.apply(null) + val result = expr.eval(null) if (result != answer) fail(s"$expr should not evaluate to $result, expected: $answer") } } @@ -105,7 +105,7 @@ class ExpressionEvaluationSuite extends FunSuite { truthTable.foreach { case (l,r,answer) => val expr = op(Literal(l, BooleanType), Literal(r, BooleanType)) - val result = expr.apply(null) + val result = expr.eval(null) if (result != answer) fail(s"$expr should not evaluate to $result, expected: $answer") } @@ -113,7 +113,7 @@ class ExpressionEvaluationSuite extends FunSuite { } def evaluate(expression: Expression, inputRow: Row = EmptyRow): Any = { - expression.apply(inputRow) + expression.eval(inputRow) } def checkEvaluation(expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala index e902e6ced521d..cff4887936ae1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala @@ -36,10 +36,10 @@ case class Generate( child: SparkPlan) extends UnaryNode { - def output = + override def output = if (join) child.output ++ generator.output else generator.output - def execute() = { + override def execute() = { if (join) { child.execute().mapPartitions { iter => val nullValues = Seq.fill(generator.output.size)(Literal(null)) @@ -52,7 +52,7 @@ case class Generate( val joinedRow = new JoinedRow iter.flatMap {row => - val outputRows = generator(row) + val outputRows = generator.eval(row) if (outer && outputRows.isEmpty) { outerProjection(row) :: Nil } else { @@ -61,7 +61,7 @@ case class Generate( } } } else { - child.execute().mapPartitions(iter => iter.flatMap(generator)) + child.execute().mapPartitions(iter => iter.flatMap(row => generator.eval(row))) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala index 2a4f7b5670457..0890faa33b507 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala @@ -144,7 +144,7 @@ case class Aggregate( var i = 0 while (i < buffer.length) { - aggregateResults(i) = buffer(i).apply(EmptyRow) + aggregateResults(i) = buffer(i).eval(EmptyRow) i += 1 } @@ -190,7 +190,7 @@ case class Aggregate( while (i < currentBuffer.length) { // Evaluating an aggregate buffer returns the result. No row is required since we // already added all rows in the group using update. - aggregateResults(i) = currentBuffer(i).apply(EmptyRow) + aggregateResults(i) = currentBuffer(i).eval(EmptyRow) i += 1 } resultProjection(joinedRow(aggregateResults, currentGroup)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 524e5022ee14b..ab2e62463764a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -41,7 +41,7 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { override def output = child.output override def execute() = child.execute().mapPartitions { iter => - iter.filter(condition.apply(_).asInstanceOf[Boolean]) + iter.filter(condition.eval(_).asInstanceOf[Boolean]) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala index ca5c8b8eb63dc..e55648b8ed15a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala @@ -39,9 +39,9 @@ case class ExampleTGF(input: Seq[Attribute] = Seq('name, 'age)) extends Generato val Seq(nameAttr, ageAttr) = input - override def apply(input: Row): TraversableOnce[Row] = { - val name = nameAttr.apply(input) - val age = ageAttr.apply(input).asInstanceOf[Int] + override def eval(input: Row): TraversableOnce[Row] = { + val name = nameAttr.eval(input) + val age = ageAttr.eval(input).asInstanceOf[Int] Iterator( new GenericRow(Array[Any](s"$name is $age years old")), diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala index e2d9d8de2572a..821fb22112f87 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala @@ -106,7 +106,7 @@ case class HiveTableScan( } private def castFromString(value: String, dataType: DataType) = { - Cast(Literal(value), dataType).apply(null) + Cast(Literal(value), dataType).eval(null) } @transient @@ -134,7 +134,7 @@ case class HiveTableScan( // Only partitioned values are needed here, since the predicate has already been bound to // partition key attribute references. val row = new GenericRow(castedValues.toArray) - shouldKeep.apply(row).asInstanceOf[Boolean] + shouldKeep.eval(row).asInstanceOf[Boolean] } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 2c607455c8e4b..f9b437d435eba 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -190,8 +190,8 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd } // TODO: Finish input output types. - override def apply(input: Row): Any = { - val evaluatedChildren = children.map(_.apply(input)) + override def eval(input: Row): Any = { + val evaluatedChildren = children.map(_.eval(input)) // Wrap the function arguments in the expected types. val args = evaluatedChildren.zip(wrappers).map { case (arg, wrapper) => wrapper(arg) @@ -216,12 +216,12 @@ case class HiveGenericUdf( val dataType: DataType = inspectorToDataType(returnInspector) - override def apply(input: Row): Any = { + override def eval(input: Row): Any = { returnInspector // Make sure initialized. val args = children.map { v => new DeferredObject { override def prepare(i: Int) = {} - override def get(): AnyRef = wrap(v.apply(input)) + override def get(): AnyRef = wrap(v.eval(input)) } }.toArray unwrap(function.evaluate(args)) @@ -406,7 +406,7 @@ case class HiveGenericUdtf( } } - override def apply(input: Row): TraversableOnce[Row] = { + override def eval(input: Row): TraversableOnce[Row] = { outputInspectors // Make sure initialized. val inputProjection = new Projection(children) @@ -460,7 +460,7 @@ case class HiveUdafFunction( private val buffer = function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] - override def apply(input: Row): Any = unwrapData(function.evaluate(buffer), returnInspector) + override def eval(input: Row): Any = unwrapData(function.evaluate(buffer), returnInspector) @transient val inputProjection = new Projection(exprs) From 9dd8b9166225cbcfcab1f39268244c2feb42a658 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 7 Apr 2014 13:28:14 -0500 Subject: [PATCH 016/641] SPARK-1252. On YARN, use container-log4j.properties for executors container-log4j.properties is a file that YARN provides so that containers can have log4j.properties distinct from that of the NodeManagers. Logs now go to syslog, and stderr and stdout just have the process's standard err and standard out. I tested this on pseudo-distributed clusters for both yarn (Hadoop 2.2) and yarn-alpha (Hadoop 0.23.7)/ Author: Sandy Ryza Closes #148 from sryza/sandy-spark-1252 and squashes the following commits: c0043b8 [Sandy Ryza] Put log4j.properties file under common 55823da [Sandy Ryza] Add license headers to new files 10934b8 [Sandy Ryza] Add log4j-spark-container.properties and support SPARK_LOG4J_CONF e74450b [Sandy Ryza] SPARK-1252. On YARN, use container-log4j.properties for executors --- .../spark/deploy/yarn/ExecutorRunnable.scala | 3 ++- .../log4j-spark-container.properties | 24 +++++++++++++++++++ .../apache/spark/deploy/yarn/ClientBase.scala | 8 +++++-- .../deploy/yarn/ExecutorRunnableUtil.scala | 7 +++++- .../deploy/yarn/YarnSparkHadoopUtil.scala | 7 ++++++ yarn/pom.xml | 6 +++++ .../spark/deploy/yarn/ExecutorRunnable.scala | 3 ++- 7 files changed, 53 insertions(+), 5 deletions(-) create mode 100644 yarn/common/src/main/resources/log4j-spark-container.properties diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 981e8b05f602d..3469b7decedf6 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -81,7 +81,8 @@ class ExecutorRunnable( credentials.writeTokenStorageToStream(dob) ctx.setContainerTokens(ByteBuffer.wrap(dob.getData())) - val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores) + val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, + localResources.contains(ClientBase.LOG4J_PROP)) logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) diff --git a/yarn/common/src/main/resources/log4j-spark-container.properties b/yarn/common/src/main/resources/log4j-spark-container.properties new file mode 100644 index 0000000000000..a1e37a0be27dd --- /dev/null +++ b/yarn/common/src/main/resources/log4j-spark-container.properties @@ -0,0 +1,24 @@ +# +# Licensed 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. See accompanying LICENSE file. + +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + +# Settings to quiet third party logs that are too verbose +log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO +log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO 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 bc267900fcf1d..eb42922aea228 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 @@ -266,11 +266,11 @@ trait ClientBase extends Logging { localResources: HashMap[String, LocalResource], stagingDir: String): HashMap[String, String] = { logInfo("Setting up the launch environment") - val log4jConfLocalRes = localResources.getOrElse(ClientBase.LOG4J_PROP, null) val env = new HashMap[String, String]() - ClientBase.populateClasspath(yarnConf, sparkConf, log4jConfLocalRes != null, env) + ClientBase.populateClasspath(yarnConf, sparkConf, localResources.contains(ClientBase.LOG4J_PROP), + env) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() @@ -344,6 +344,10 @@ trait ClientBase extends Logging { JAVA_OPTS += " " + env("SPARK_JAVA_OPTS") } + if (!localResources.contains(ClientBase.LOG4J_PROP)) { + JAVA_OPTS += " " + YarnSparkHadoopUtil.getLoggingArgsForContainerCommandLine() + } + // Command for the ApplicationMaster val commands = List[String]( Environment.JAVA_HOME.$() + "/bin/java" + diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 2079697d8160e..b3696c5fe7183 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -50,7 +50,8 @@ trait ExecutorRunnableUtil extends Logging { slaveId: String, hostname: String, executorMemory: Int, - executorCores: Int) = { + executorCores: Int, + userSpecifiedLogFile: Boolean) = { // Extra options for the JVM var JAVA_OPTS = "" // Set the JVM memory @@ -63,6 +64,10 @@ trait ExecutorRunnableUtil extends Logging { JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " " + if (!userSpecifiedLogFile) { + JAVA_OPTS += " " + YarnSparkHadoopUtil.getLoggingArgsForContainerCommandLine() + } + // Commenting it out for now - so that people can refer to the properties if required. Remove // it once cpuset version is pushed out. // The context is, default gc for server class machines end up using all cores to do gc - hence diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 4c6e1dcd6dac3..314a7550ada71 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.conf.Configuration import org.apache.spark.deploy.SparkHadoopUtil @@ -67,3 +68,9 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } } + +object YarnSparkHadoopUtil { + def getLoggingArgsForContainerCommandLine(): String = { + "-Dlog4j.configuration=log4j-spark-container.properties" + } +} diff --git a/yarn/pom.xml b/yarn/pom.xml index 35e31760c1f02..3342cb65edcd1 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -167,6 +167,12 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + + ../common/src/main/resources + + diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 53c403f7d0913..81d9d1b5c9280 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -78,7 +78,8 @@ class ExecutorRunnable( credentials.writeTokenStorageToStream(dob) ctx.setTokens(ByteBuffer.wrap(dob.getData())) - val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores) + val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, + localResources.contains(ClientBase.LOG4J_PROP)) logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) From 2a2ca48be61ed0d72c4347e1c042a264b94db3e8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 7 Apr 2014 12:47:27 -0700 Subject: [PATCH 017/641] HOTFIX: Disable actor input stream test. This test makes incorrect assumptions about the behavior of Thread.sleep(). Author: Patrick Wendell Closes #347 from pwendell/stream-tests and squashes the following commits: 10e09e0 [Patrick Wendell] HOTFIX: Disable actor input stream. --- .../scala/org/apache/spark/streaming/InputStreamsSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 7df206241beb6..389b23d4d5e4b 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -144,8 +144,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } - - test("actor input stream") { + // TODO: This test makes assumptions about Thread.sleep() and is flaky + ignore("actor input stream") { // Start the server val testServer = new TestServer() val port = testServer.port From 0307db0f55b714930c7ea118d5451190ea8c1a94 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 7 Apr 2014 13:06:30 -0700 Subject: [PATCH 018/641] SPARK-1099: Introduce local[*] mode to infer number of cores This is the default mode for running spark-shell and pyspark, intended to allow users running spark for the first time to see the performance benefits of using multiple cores, while not breaking backwards compatibility for users who use "local" mode and expect exactly 1 core. Author: Aaron Davidson Closes #182 from aarondav/110 and squashes the following commits: a88294c [Aaron Davidson] Rebased changes for new spark-shell a9f393e [Aaron Davidson] SPARK-1099: Introduce local[*] mode to infer number of cores --- bin/spark-shell | 4 ++-- core/src/main/scala/org/apache/spark/SparkContext.scala | 9 ++++++--- .../spark/SparkContextSchedulerCreationSuite.scala | 8 ++++++++ docs/python-programming-guide.md | 7 ++++--- docs/scala-programming-guide.md | 5 +++-- python/pyspark/shell.py | 2 +- .../main/scala/org/apache/spark/repl/SparkILoop.scala | 2 +- 7 files changed, 25 insertions(+), 12 deletions(-) diff --git a/bin/spark-shell b/bin/spark-shell index 535ee3ccd8269..ea12d256b23a1 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -34,7 +34,7 @@ set -o posix FWDIR="$(cd `dirname $0`/..; pwd)" SPARK_REPL_OPTS="${SPARK_REPL_OPTS:-""}" -DEFAULT_MASTER="local" +DEFAULT_MASTER="local[*]" MASTER=${MASTER:-""} info_log=0 @@ -64,7 +64,7 @@ ${txtbld}OPTIONS${txtrst}: is followed by m for megabytes or g for gigabytes, e.g. "1g". -dm --driver-memory : The memory used by the Spark Shell, the number is followed by m for megabytes or g for gigabytes, e.g. "1g". - -m --master : A full string that describes the Spark Master, defaults to "local" + -m --master : A full string that describes the Spark Master, defaults to "local[*]" e.g. "spark://localhost:7077". --log-conf : Enables logging of the supplied SparkConf as INFO at start of the Spark Context. diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 8382dd44f3484..e5ebd350eeced 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1285,8 +1285,8 @@ object SparkContext extends Logging { /** Creates a task scheduler based on a given master URL. Extracted for testing. */ private def createTaskScheduler(sc: SparkContext, master: String): TaskScheduler = { - // Regular expression used for local[N] master format - val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r + // Regular expression used for local[N] and local[*] master formats + val LOCAL_N_REGEX = """local\[([0-9\*]+)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+)\s*,\s*([0-9]+)\]""".r // Regular expression for simulating a Spark cluster of [N, cores, memory] locally @@ -1309,8 +1309,11 @@ object SparkContext extends Logging { scheduler case LOCAL_N_REGEX(threads) => + def localCpuCount = Runtime.getRuntime.availableProcessors() + // local[*] estimates the number of cores on the machine; local[N] uses exactly N threads. + val threadCount = if (threads == "*") localCpuCount else threads.toInt val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalBackend(scheduler, threads.toInt) + val backend = new LocalBackend(scheduler, threadCount) scheduler.initialize(backend) scheduler diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index b543471a5d35b..94fba102865b3 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -51,6 +51,14 @@ class SparkContextSchedulerCreationSuite } } + test("local-*") { + val sched = createTaskScheduler("local[*]") + sched.backend match { + case s: LocalBackend => assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case _ => fail() + } + } + test("local-n") { val sched = createTaskScheduler("local[5]") assert(sched.maxTaskFailures === 1) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index c2e5327324898..888631e7025b0 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -82,15 +82,16 @@ The Python shell can be used explore data interactively and is a simple way to l >>> help(pyspark) # Show all pyspark functions {% endhighlight %} -By default, the `bin/pyspark` shell creates SparkContext that runs applications locally on a single core. -To connect to a non-local cluster, or use multiple cores, set the `MASTER` environment variable. +By default, the `bin/pyspark` shell creates SparkContext that runs applications locally on all of +your machine's logical cores. +To connect to a non-local cluster, or to specify a number of cores, set the `MASTER` environment variable. For example, to use the `bin/pyspark` shell with a [standalone Spark cluster](spark-standalone.html): {% highlight bash %} $ MASTER=spark://IP:PORT ./bin/pyspark {% endhighlight %} -Or, to use four cores on the local machine: +Or, to use exactly four cores on the local machine: {% highlight bash %} $ MASTER=local[4] ./bin/pyspark diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 77373890eead7..a07cd2e0a32a2 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -54,7 +54,7 @@ object for more advanced configuration. The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later. -In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on four cores, use +In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on exactly four cores, use {% highlight bash %} $ MASTER=local[4] ./bin/spark-shell @@ -74,6 +74,7 @@ The master URL passed to Spark can be in one of the following formats: Master URLMeaning local Run Spark locally with one worker thread (i.e. no parallelism at all). local[K] Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine). + local[*] Run Spark locally with as many worker threads as logical cores on your machine. spark://HOST:PORT Connect to the given Spark standalone cluster master. The port must be whichever one your master is configured to use, which is 7077 by default. @@ -84,7 +85,7 @@ The master URL passed to Spark can be in one of the following formats: -If no master URL is specified, the spark shell defaults to "local". +If no master URL is specified, the spark shell defaults to "local[*]". For running on YARN, Spark launches an instance of the standalone deploy cluster within YARN; see [running on YARN](running-on-yarn.html) for details. diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 3d779faf1fa44..35e48276e3cb9 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -29,7 +29,7 @@ # this is the equivalent of ADD_JARS add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("ADD_FILES") != None else None -sc = SparkContext(os.environ.get("MASTER", "local"), "PySparkShell", pyFiles=add_files) +sc = SparkContext(os.environ.get("MASTER", "local[*]"), "PySparkShell", pyFiles=add_files) print """Welcome to ____ __ diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 9b1da195002c2..5a367b6bb79de 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -963,7 +963,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, case Some(m) => m case None => { val prop = System.getenv("MASTER") - if (prop != null) prop else "local" + if (prop != null) prop else "local[*]" } } master From 14c9238aa7173ba663a999ef320d8cffb73306c4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 7 Apr 2014 18:38:44 -0700 Subject: [PATCH 019/641] [sql] Rename execution/aggregates.scala Aggregate.scala, and added a bunch of private[this] to variables. Author: Reynold Xin Closes #348 from rxin/aggregate and squashes the following commits: f4bc36f [Reynold Xin] Rename execution/aggregates.scala Aggregate.scala, and added a bunch of private[this] to variables. --- .../{aggregates.scala => Aggregate.scala} | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/{aggregates.scala => Aggregate.scala} (92%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala similarity index 92% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala index 0890faa33b507..3a4f071eebedf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala @@ -56,9 +56,9 @@ case class Aggregate( // HACK: Generators don't correctly preserve their output through serializations so we grab // out child's output attributes statically here. - val childOutput = child.output + private[this] val childOutput = child.output - def output = aggregateExpressions.map(_.toAttribute) + override def output = aggregateExpressions.map(_.toAttribute) /** * An aggregate that needs to be computed for each row in a group. @@ -75,7 +75,7 @@ case class Aggregate( /** A list of aggregates that need to be computed for each group. */ @transient - lazy val computedAggregates = aggregateExpressions.flatMap { agg => + private[this] lazy val computedAggregates = aggregateExpressions.flatMap { agg => agg.collect { case a: AggregateExpression => ComputedAggregate( @@ -87,10 +87,10 @@ case class Aggregate( /** The schema of the result of all aggregate evaluations */ @transient - lazy val computedSchema = computedAggregates.map(_.resultAttribute) + private[this] lazy val computedSchema = computedAggregates.map(_.resultAttribute) /** Creates a new aggregate buffer for a group. */ - def newAggregateBuffer(): Array[AggregateFunction] = { + private[this] def newAggregateBuffer(): Array[AggregateFunction] = { val buffer = new Array[AggregateFunction](computedAggregates.length) var i = 0 while (i < computedAggregates.length) { @@ -102,7 +102,7 @@ case class Aggregate( /** Named attributes used to substitute grouping attributes into the final result. */ @transient - lazy val namedGroups = groupingExpressions.map { + private[this] lazy val namedGroups = groupingExpressions.map { case ne: NamedExpression => ne -> ne.toAttribute case e => e -> Alias(e, s"groupingExpr:$e")().toAttribute } @@ -112,7 +112,7 @@ case class Aggregate( * expression into the final result expression. */ @transient - lazy val resultMap = + private[this] lazy val resultMap = (computedAggregates.map { agg => agg.unbound -> agg.resultAttribute} ++ namedGroups).toMap /** @@ -120,13 +120,13 @@ case class Aggregate( * output rows given a group and the result of all aggregate computations. */ @transient - lazy val resultExpressions = aggregateExpressions.map { agg => + private[this] lazy val resultExpressions = aggregateExpressions.map { agg => agg.transform { case e: Expression if resultMap.contains(e) => resultMap(e) } } - def execute() = attachTree(this, "execute") { + override def execute() = attachTree(this, "execute") { if (groupingExpressions.isEmpty) { child.execute().mapPartitions { iter => val buffer = newAggregateBuffer() From 55dfd5dcdbf3a9bfddb2108c8325bda3100eb33d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 7 Apr 2014 18:39:18 -0700 Subject: [PATCH 020/641] Removed the default eval implementation from Expression, and added a bunch of override's in classes I touched. It is more robust to not provide a default implementation for Expression's. Author: Reynold Xin Closes #350 from rxin/eval-default and squashes the following commits: 0a83b8f [Reynold Xin] Removed the default eval implementation from Expression, and added a bunch of override's in classes I touched. --- .../sql/catalyst/analysis/unresolved.scala | 52 ++++++++++++------- .../sql/catalyst/expressions/Expression.scala | 3 +- .../sql/catalyst/expressions/SortOrder.scala | 11 +++- .../sql/catalyst/expressions/aggregates.scala | 8 +++ .../expressions/namedExpressions.scala | 21 +++++--- .../plans/physical/partitioning.scala | 32 ++++++++---- .../ExpressionEvaluationSuite.scala | 5 +- .../optimizer/ConstantFoldingSuite.scala | 2 +- 8 files changed, 89 insertions(+), 45 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 41e9bcef3cd7f..d629172a7426e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.{errors, trees} -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression} +import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.BaseRelation import org.apache.spark.sql.catalyst.trees.TreeNode @@ -36,7 +37,7 @@ case class UnresolvedRelation( databaseName: Option[String], tableName: String, alias: Option[String] = None) extends BaseRelation { - def output = Nil + override def output = Nil override lazy val resolved = false } @@ -44,26 +45,33 @@ case class UnresolvedRelation( * Holds the name of an attribute that has yet to be resolved. */ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNode[Expression] { - def exprId = throw new UnresolvedException(this, "exprId") - def dataType = throw new UnresolvedException(this, "dataType") - def nullable = throw new UnresolvedException(this, "nullable") - def qualifiers = throw new UnresolvedException(this, "qualifiers") + override def exprId = throw new UnresolvedException(this, "exprId") + override def dataType = throw new UnresolvedException(this, "dataType") + override def nullable = throw new UnresolvedException(this, "nullable") + override def qualifiers = throw new UnresolvedException(this, "qualifiers") override lazy val resolved = false - def newInstance = this - def withQualifiers(newQualifiers: Seq[String]) = this + override def newInstance = this + override def withQualifiers(newQualifiers: Seq[String]) = this + + // Unresolved attributes are transient at compile time and don't get evaluated during execution. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") override def toString: String = s"'$name" } case class UnresolvedFunction(name: String, children: Seq[Expression]) extends Expression { - def exprId = throw new UnresolvedException(this, "exprId") - def dataType = throw new UnresolvedException(this, "dataType") + override def dataType = throw new UnresolvedException(this, "dataType") override def foldable = throw new UnresolvedException(this, "foldable") - def nullable = throw new UnresolvedException(this, "nullable") - def qualifiers = throw new UnresolvedException(this, "qualifiers") - def references = children.flatMap(_.references).toSet + override def nullable = throw new UnresolvedException(this, "nullable") + override def references = children.flatMap(_.references).toSet override lazy val resolved = false + + // Unresolved functions are transient at compile time and don't get evaluated during execution. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + override def toString = s"'$name(${children.mkString(",")})" } @@ -79,15 +87,15 @@ case class Star( mapFunction: Attribute => Expression = identity[Attribute]) extends Attribute with trees.LeafNode[Expression] { - def name = throw new UnresolvedException(this, "exprId") - def exprId = throw new UnresolvedException(this, "exprId") - def dataType = throw new UnresolvedException(this, "dataType") - def nullable = throw new UnresolvedException(this, "nullable") - def qualifiers = throw new UnresolvedException(this, "qualifiers") + override def name = throw new UnresolvedException(this, "exprId") + override def exprId = throw new UnresolvedException(this, "exprId") + override def dataType = throw new UnresolvedException(this, "dataType") + override def nullable = throw new UnresolvedException(this, "nullable") + override def qualifiers = throw new UnresolvedException(this, "qualifiers") override lazy val resolved = false - def newInstance = this - def withQualifiers(newQualifiers: Seq[String]) = this + override def newInstance = this + override def withQualifiers(newQualifiers: Seq[String]) = this def expand(input: Seq[Attribute]): Seq[NamedExpression] = { val expandedAttributes: Seq[Attribute] = table match { @@ -104,5 +112,9 @@ case class Star( mappedAttributes } + // Star gets expanded at runtime so we never evaluate a Star. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + override def toString = table.map(_ + ".").getOrElse("") + "*" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index f190bd0cca375..8a1db8e796816 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -50,8 +50,7 @@ abstract class Expression extends TreeNode[Expression] { def references: Set[Attribute] /** Returns the result of evaluating this expression on a given input Row */ - def eval(input: Row = null): EvaluatedType = - throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + def eval(input: Row = null): EvaluatedType /** * Returns `true` if this expression and all its children have been resolved to a specific schema diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala index d5d93778f4b8d..08b2f11d20f5e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.errors.TreeNodeException + abstract sealed class SortDirection case object Ascending extends SortDirection case object Descending extends SortDirection @@ -26,7 +28,12 @@ case object Descending extends SortDirection * transformations over expression will descend into its child. */ case class SortOrder(child: Expression, direction: SortDirection) extends UnaryExpression { - def dataType = child.dataType - def nullable = child.nullable + override def dataType = child.dataType + override def nullable = child.nullable + + // SortOrder itself is never evaluated. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + override def toString = s"$child ${if (direction == Ascending) "ASC" else "DESC"}" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 5edcea14278c7..b152f95f96c70 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.catalyst.errors.TreeNodeException abstract class AggregateExpression extends Expression { self: Product => @@ -28,6 +29,13 @@ abstract class AggregateExpression extends Expression { * of input rows/ */ def newInstance(): AggregateFunction + + /** + * [[AggregateExpression.eval]] should never be invoked because [[AggregateExpression]]'s are + * replaced with a physical aggregate operator at runtime. + */ + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index eb4bc8e755284..a8145c37c20fa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.types._ object NamedExpression { @@ -58,9 +59,9 @@ abstract class Attribute extends NamedExpression { def withQualifiers(newQualifiers: Seq[String]): Attribute - def references = Set(this) def toAttribute = this def newInstance: Attribute + override def references = Set(this) } /** @@ -77,15 +78,15 @@ case class Alias(child: Expression, name: String) (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) extends NamedExpression with trees.UnaryNode[Expression] { - type EvaluatedType = Any + override type EvaluatedType = Any override def eval(input: Row) = child.eval(input) - def dataType = child.dataType - def nullable = child.nullable - def references = child.references + override def dataType = child.dataType + override def nullable = child.nullable + override def references = child.references - def toAttribute = { + override def toAttribute = { if (resolved) { AttributeReference(name, child.dataType, child.nullable)(exprId, qualifiers) } else { @@ -127,7 +128,7 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea h } - def newInstance = AttributeReference(name, dataType, nullable)(qualifiers = qualifiers) + override def newInstance = AttributeReference(name, dataType, nullable)(qualifiers = qualifiers) /** * Returns a copy of this [[AttributeReference]] with changed nullability. @@ -143,7 +144,7 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea /** * Returns a copy of this [[AttributeReference]] with new qualifiers. */ - def withQualifiers(newQualifiers: Seq[String]) = { + override def withQualifiers(newQualifiers: Seq[String]) = { if (newQualifiers == qualifiers) { this } else { @@ -151,5 +152,9 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea } } + // Unresolved attributes are transient at compile time and don't get evaluated during execution. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + override def toString: String = s"$name#${exprId.id}$typeSuffix" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 8893744eb2e7a..ffb3a92f8f340 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.catalyst.plans.physical -import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder} +import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.expressions.{Expression, Row, SortOrder} import org.apache.spark.sql.catalyst.types.IntegerType /** @@ -139,12 +140,12 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) extends Expression with Partitioning { - def children = expressions - def references = expressions.flatMap(_.references).toSet - def nullable = false - def dataType = IntegerType + override def children = expressions + override def references = expressions.flatMap(_.references).toSet + override def nullable = false + override def dataType = IntegerType - lazy val clusteringSet = expressions.toSet + private[this] lazy val clusteringSet = expressions.toSet override def satisfies(required: Distribution): Boolean = required match { case UnspecifiedDistribution => true @@ -158,6 +159,9 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) case h: HashPartitioning if h == this => true case _ => false } + + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } /** @@ -168,17 +172,20 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) * partition. * - Each partition will have a `min` and `max` row, relative to the given ordering. All rows * that are in between `min` and `max` in this `ordering` will reside in this partition. + * + * This class extends expression primarily so that transformations over expression will descend + * into its child. */ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) extends Expression with Partitioning { - def children = ordering - def references = ordering.flatMap(_.references).toSet - def nullable = false - def dataType = IntegerType + override def children = ordering + override def references = ordering.flatMap(_.references).toSet + override def nullable = false + override def dataType = IntegerType - lazy val clusteringSet = ordering.map(_.child).toSet + private[this] lazy val clusteringSet = ordering.map(_.child).toSet override def satisfies(required: Distribution): Boolean = required match { case UnspecifiedDistribution => true @@ -195,4 +202,7 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) case r: RangePartitioning if r == this => true case _ => false } + + override def eval(input: Row): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 92987405aa313..31be6c4ef1b0b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -100,7 +100,10 @@ class ExpressionEvaluationSuite extends FunSuite { (null, false, null) :: (null, null, null) :: Nil) - def booleanLogicTest(name: String, op: (Expression, Expression) => Expression, truthTable: Seq[(Any, Any, Any)]) { + def booleanLogicTest( + name: String, + op: (Expression, Expression) => Expression, + truthTable: Seq[(Any, Any, Any)]) { test(s"3VL $name") { truthTable.foreach { case (l,r,answer) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index 2ab14f48ccc8a..20dfba847790c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.catalyst.types.IntegerType +import org.apache.spark.sql.catalyst.types.{DoubleType, IntegerType} // For implicit conversions import org.apache.spark.sql.catalyst.dsl.plans._ From 31e6fff03730bb915a836d77dcd43d098afd1dbd Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 7 Apr 2014 18:40:08 -0700 Subject: [PATCH 021/641] Added eval for Rand (without any support for user-defined seed). Author: Reynold Xin Closes #349 from rxin/rand and squashes the following commits: fd11322 [Reynold Xin] Added eval for Rand (without any support for user-defined seed). --- .../spark/sql/catalyst/expressions/Rand.scala | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala index 0bde621602944..38f836f0a1a0e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala @@ -17,11 +17,18 @@ package org.apache.spark.sql.catalyst.expressions +import java.util.Random import org.apache.spark.sql.catalyst.types.DoubleType + case object Rand extends LeafExpression { - def dataType = DoubleType - def nullable = false - def references = Set.empty + override def dataType = DoubleType + override def nullable = false + override def references = Set.empty + + private[this] lazy val rand = new Random + + override def eval(input: Row = null) = rand.nextDouble().asInstanceOf[EvaluatedType] + override def toString = "RAND()" } From f27e56aa612538188a8550fe72ee20b8b13304d7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 7 Apr 2014 19:28:24 -0700 Subject: [PATCH 022/641] Change timestamp cast semantics. When cast to numeric types, return the unix time in seconds (instead of millis). @marmbrus @chenghao-intel Author: Reynold Xin Closes #352 from rxin/timestamp-cast and squashes the following commits: 18aacd3 [Reynold Xin] Fixed precision for double. 2adb235 [Reynold Xin] Change timestamp cast semantics. When cast to numeric types, return the unix time in seconds (instead of millis). --- .../spark/sql/catalyst/dsl/package.scala | 2 +- .../spark/sql/catalyst/expressions/Cast.scala | 23 ++++++++++------ .../ExpressionEvaluationSuite.scala | 27 ++++++++++++++++--- 3 files changed, 40 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 2d62e4cbbce01..987befe8e22ee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -104,7 +104,7 @@ package object dsl { implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s = sym.name } // TODO more implicit class for literal? implicit class DslString(val s: String) extends ImplicitOperators { - def expr: Expression = Literal(s) + override def expr: Expression = Literal(s) def attr = analysis.UnresolvedAttribute(s) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 89226999ca005..17118499d0c87 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -87,7 +87,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { private def decimalToTimestamp(d: BigDecimal) = { val seconds = d.longValue() - val bd = (d - seconds) * (1000000000) + val bd = (d - seconds) * 1000000000 val nanos = bd.intValue() // Convert to millis @@ -96,18 +96,23 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { // remaining fractional portion as nanos t.setNanos(nanos) - t } - private def timestampToDouble(t: Timestamp) = (t.getSeconds() + t.getNanos().toDouble / 1000) + // Timestamp to long, converting milliseconds to seconds + private def timestampToLong(ts: Timestamp) = ts.getTime / 1000 + + private def timestampToDouble(ts: Timestamp) = { + // First part is the seconds since the beginning of time, followed by nanosecs. + ts.getTime / 1000 + ts.getNanos.toDouble / 1000000000 + } def castToLong: Any => Any = child.dataType match { case StringType => nullOrCast[String](_, s => try s.toLong catch { case _: NumberFormatException => null }) case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) - case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toLong) + case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t)) case DecimalType => nullOrCast[BigDecimal](_, _.toLong) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toLong(b) } @@ -117,7 +122,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case _: NumberFormatException => null }) case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) - case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toInt) + case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t).toInt) case DecimalType => nullOrCast[BigDecimal](_, _.toInt) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b) } @@ -127,7 +132,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case _: NumberFormatException => null }) case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) - case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toShort) + case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t).toShort) case DecimalType => nullOrCast[BigDecimal](_, _.toShort) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toShort } @@ -137,7 +142,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case _: NumberFormatException => null }) case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) - case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toByte) + case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t).toByte) case DecimalType => nullOrCast[BigDecimal](_, _.toByte) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte } @@ -147,7 +152,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case _: NumberFormatException => null }) case BooleanType => nullOrCast[Boolean](_, b => if(b) BigDecimal(1) else BigDecimal(0)) - case TimestampType => nullOrCast[Timestamp](_, t => BigDecimal(timestampToDouble(t))) + case TimestampType => + // Note that we lose precision here. + nullOrCast[Timestamp](_, t => BigDecimal(timestampToDouble(t))) case x: NumericType => b => BigDecimal(x.numeric.asInstanceOf[Numeric[Any]].toDouble(b)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 31be6c4ef1b0b..888a19d79f7e4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -201,7 +201,7 @@ class ExpressionEvaluationSuite extends FunSuite { val sts = "1970-01-01 00:00:01.0" val ts = Timestamp.valueOf(sts) - + checkEvaluation("abdef" cast StringType, "abdef") checkEvaluation("abdef" cast DecimalType, null) checkEvaluation("abdef" cast TimestampType, null) @@ -209,7 +209,6 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Literal(1) cast LongType, 1) checkEvaluation(Cast(Literal(1) cast TimestampType, LongType), 1) - checkEvaluation(Cast(Literal(BigDecimal(1)) cast TimestampType, DecimalType), 1) checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble) checkEvaluation(Cast(Literal(sts) cast TimestampType, StringType), sts) @@ -240,12 +239,34 @@ class ExpressionEvaluationSuite extends FunSuite { intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)} } - + test("timestamp") { val ts1 = new Timestamp(12) val ts2 = new Timestamp(123) checkEvaluation(Literal("ab") < Literal("abc"), true) checkEvaluation(Literal(ts1) < Literal(ts2), true) } + + test("timestamp casting") { + val millis = 15 * 1000 + 2 + val ts = new Timestamp(millis) + val ts1 = new Timestamp(15 * 1000) // a timestamp without the milliseconds part + checkEvaluation(Cast(ts, ShortType), 15) + checkEvaluation(Cast(ts, IntegerType), 15) + checkEvaluation(Cast(ts, LongType), 15) + checkEvaluation(Cast(ts, FloatType), 15.002f) + checkEvaluation(Cast(ts, DoubleType), 15.002) + checkEvaluation(Cast(Cast(ts, ShortType), TimestampType), ts1) + checkEvaluation(Cast(Cast(ts, IntegerType), TimestampType), ts1) + checkEvaluation(Cast(Cast(ts, LongType), TimestampType), ts1) + checkEvaluation(Cast(Cast(millis.toFloat / 1000, TimestampType), FloatType), + millis.toFloat / 1000) + checkEvaluation(Cast(Cast(millis.toDouble / 1000, TimestampType), DoubleType), + millis.toDouble / 1000) + checkEvaluation(Cast(Literal(BigDecimal(1)) cast TimestampType, DecimalType), 1) + + // A test for higher precision than millis + checkEvaluation(Cast(Cast(0.00000001, TimestampType), DoubleType), 0.00000001) + } } From 0d0493fcf7fc86d30b0ddd4e2c5a293c5c88eb9d Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 7 Apr 2014 22:24:12 -0700 Subject: [PATCH 023/641] [SPARK-1402] Added 3 more compression schemes JIRA issue: [SPARK-1402](https://issues.apache.org/jira/browse/SPARK-1402) This PR provides 3 more compression schemes for Spark SQL in-memory columnar storage: * `BooleanBitSet` * `IntDelta` * `LongDelta` Now there are 6 compression schemes in total, including the no-op `PassThrough` scheme. Also fixed a bug in PR #286: not all compression schemes are added as available schemes when accessing an in-memory column, and when a column is compressed with an unrecognised scheme, `ColumnAccessor` throws exception. Author: Cheng Lian Closes #330 from liancheng/moreCompressionSchemes and squashes the following commits: 1d037b8 [Cheng Lian] Fixed SPARK-1436: in-memory column byte buffer must be able to be accessed multiple times d7c0e8f [Cheng Lian] Added test suite for IntegralDelta (IntDelta & LongDelta) 3c1ad7a [Cheng Lian] Added test suite for BooleanBitSet, refactored other test suites 44fe4b2 [Cheng Lian] Refactored CompressionScheme, added 3 more compression schemes. --- .../spark/sql/columnar/ColumnAccessor.scala | 23 +- .../spark/sql/columnar/ColumnStats.scala | 6 + .../CompressibleColumnBuilder.scala | 6 +- .../compression/CompressionScheme.scala | 28 +- .../compression/compressionSchemes.scala | 266 +++++++++++++++--- .../sql/columnar/ColumnarQuerySuite.scala | 8 + .../compression/BooleanBitSetSuite.scala | 98 +++++++ .../compression/DictionaryEncodingSuite.scala | 122 ++++---- .../compression/IntegralDeltaSuite.scala | 115 ++++++++ .../compression/RunLengthEncodingSuite.scala | 87 +++--- .../TestCompressibleColumnBuilder.scala | 6 +- 11 files changed, 586 insertions(+), 179 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index ffd4894b5213d..3c39e1d350fa8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -100,20 +100,21 @@ private[sql] class GenericColumnAccessor(buffer: ByteBuffer) private[sql] object ColumnAccessor { def apply(buffer: ByteBuffer): ColumnAccessor = { + val dup = buffer.duplicate().order(ByteOrder.nativeOrder) // The first 4 bytes in the buffer indicate the column type. - val columnTypeId = buffer.getInt() + val columnTypeId = dup.getInt() columnTypeId match { - case INT.typeId => new IntColumnAccessor(buffer) - case LONG.typeId => new LongColumnAccessor(buffer) - case FLOAT.typeId => new FloatColumnAccessor(buffer) - case DOUBLE.typeId => new DoubleColumnAccessor(buffer) - case BOOLEAN.typeId => new BooleanColumnAccessor(buffer) - case BYTE.typeId => new ByteColumnAccessor(buffer) - case SHORT.typeId => new ShortColumnAccessor(buffer) - case STRING.typeId => new StringColumnAccessor(buffer) - case BINARY.typeId => new BinaryColumnAccessor(buffer) - case GENERIC.typeId => new GenericColumnAccessor(buffer) + case INT.typeId => new IntColumnAccessor(dup) + case LONG.typeId => new LongColumnAccessor(dup) + case FLOAT.typeId => new FloatColumnAccessor(dup) + case DOUBLE.typeId => new DoubleColumnAccessor(dup) + case BOOLEAN.typeId => new BooleanColumnAccessor(dup) + case BYTE.typeId => new ByteColumnAccessor(dup) + case SHORT.typeId => new ShortColumnAccessor(dup) + case STRING.typeId => new StringColumnAccessor(dup) + case BINARY.typeId => new BinaryColumnAccessor(dup) + case GENERIC.typeId => new GenericColumnAccessor(dup) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 30c6bdc7912fc..95602d321dc6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -20,6 +20,12 @@ package org.apache.spark.sql.columnar import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.types._ +/** + * Used to collect statistical information when building in-memory columns. + * + * NOTE: we intentionally avoid using `Ordering[T]` to compare values here because `Ordering[T]` + * brings significant performance penalty. + */ private[sql] sealed abstract class ColumnStats[T <: DataType, JvmType] extends Serializable { /** * Closed lower bound of this column. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index 3ac4b358ddf83..fd3b1adf9687a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -47,9 +47,9 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] import CompressionScheme._ - val compressionEncoders = schemes.filter(_.supports(columnType)).map(_.encoder) + val compressionEncoders = schemes.filter(_.supports(columnType)).map(_.encoder[T]) - protected def isWorthCompressing(encoder: Encoder) = { + protected def isWorthCompressing(encoder: Encoder[T]) = { encoder.compressionRatio < 0.8 } @@ -70,7 +70,7 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] abstract override def build() = { val rawBuffer = super.build() - val encoder = { + val encoder: Encoder[T] = { val candidate = compressionEncoders.minBy(_.compressionRatio) if (isWorthCompressing(candidate)) candidate else PassThrough.encoder } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala index d3a4ac8df926b..c605a8e4434e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala @@ -22,10 +22,8 @@ import java.nio.ByteBuffer import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnType, NativeColumnType} -private[sql] trait Encoder { - def gatherCompressibilityStats[T <: NativeType]( - value: T#JvmType, - columnType: ColumnType[T, T#JvmType]) {} +private[sql] trait Encoder[T <: NativeType] { + def gatherCompressibilityStats(value: T#JvmType, columnType: NativeColumnType[T]) {} def compressedSize: Int @@ -35,10 +33,7 @@ private[sql] trait Encoder { if (uncompressedSize > 0) compressedSize.toDouble / uncompressedSize else 1.0 } - def compress[T <: NativeType]( - from: ByteBuffer, - to: ByteBuffer, - columnType: ColumnType[T, T#JvmType]): ByteBuffer + def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]): ByteBuffer } private[sql] trait Decoder[T <: NativeType] extends Iterator[T#JvmType] @@ -48,7 +43,7 @@ private[sql] trait CompressionScheme { def supports(columnType: ColumnType[_, _]): Boolean - def encoder: Encoder + def encoder[T <: NativeType]: Encoder[T] def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T] } @@ -58,15 +53,18 @@ private[sql] trait WithCompressionSchemes { } private[sql] trait AllCompressionSchemes extends WithCompressionSchemes { - override val schemes: Seq[CompressionScheme] = { - Seq(PassThrough, RunLengthEncoding, DictionaryEncoding) - } + override val schemes: Seq[CompressionScheme] = CompressionScheme.all } private[sql] object CompressionScheme { - def apply(typeId: Int): CompressionScheme = typeId match { - case PassThrough.typeId => PassThrough - case _ => throw new UnsupportedOperationException() + val all: Seq[CompressionScheme] = + Seq(PassThrough, RunLengthEncoding, DictionaryEncoding, BooleanBitSet, IntDelta, LongDelta) + + private val typeIdToScheme = all.map(scheme => scheme.typeId -> scheme).toMap + + def apply(typeId: Int): CompressionScheme = { + typeIdToScheme.getOrElse(typeId, throw new UnsupportedOperationException( + s"Unrecognized compression scheme type ID: $typeId")) } def copyColumnHeader(from: ByteBuffer, to: ByteBuffer) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index dc2c153faf8ad..df8220b556edd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -24,7 +24,7 @@ import scala.reflect.ClassTag import scala.reflect.runtime.universe.runtimeMirror import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar._ private[sql] case object PassThrough extends CompressionScheme { @@ -32,22 +32,18 @@ private[sql] case object PassThrough extends CompressionScheme { override def supports(columnType: ColumnType[_, _]) = true - override def encoder = new this.Encoder + override def encoder[T <: NativeType] = new this.Encoder[T] override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { new this.Decoder(buffer, columnType) } - class Encoder extends compression.Encoder { + class Encoder[T <: NativeType] extends compression.Encoder[T] { override def uncompressedSize = 0 override def compressedSize = 0 - override def compress[T <: NativeType]( - from: ByteBuffer, - to: ByteBuffer, - columnType: ColumnType[T, T#JvmType]) = { - + override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]) = { // Writes compression type ID and copies raw contents to.putInt(PassThrough.typeId).put(from).rewind() to @@ -64,9 +60,9 @@ private[sql] case object PassThrough extends CompressionScheme { } private[sql] case object RunLengthEncoding extends CompressionScheme { - override def typeId = 1 + override val typeId = 1 - override def encoder = new this.Encoder + override def encoder[T <: NativeType] = new this.Encoder[T] override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { new this.Decoder(buffer, columnType) @@ -77,7 +73,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { case _ => false } - class Encoder extends compression.Encoder { + class Encoder[T <: NativeType] extends compression.Encoder[T] { private var _uncompressedSize = 0 private var _compressedSize = 0 @@ -89,10 +85,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { override def compressedSize = _compressedSize - override def gatherCompressibilityStats[T <: NativeType]( - value: T#JvmType, - columnType: ColumnType[T, T#JvmType]) { - + override def gatherCompressibilityStats(value: T#JvmType, columnType: NativeColumnType[T]) { val actualSize = columnType.actualSize(value) _uncompressedSize += actualSize @@ -111,11 +104,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { } } - override def compress[T <: NativeType]( - from: ByteBuffer, - to: ByteBuffer, - columnType: ColumnType[T, T#JvmType]) = { - + override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]) = { to.putInt(RunLengthEncoding.typeId) if (from.hasRemaining) { @@ -172,23 +161,23 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { } private[sql] case object DictionaryEncoding extends CompressionScheme { - override def typeId: Int = 2 + override val typeId = 2 // 32K unique values allowed - private val MAX_DICT_SIZE = Short.MaxValue - 1 + val MAX_DICT_SIZE = Short.MaxValue override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { - new this.Decoder[T](buffer, columnType) + new this.Decoder(buffer, columnType) } - override def encoder = new this.Encoder + override def encoder[T <: NativeType] = new this.Encoder[T] override def supports(columnType: ColumnType[_, _]) = columnType match { case INT | LONG | STRING => true case _ => false } - class Encoder extends compression.Encoder{ + class Encoder[T <: NativeType] extends compression.Encoder[T] { // Size of the input, uncompressed, in bytes. Note that we only count until the dictionary // overflows. private var _uncompressedSize = 0 @@ -201,7 +190,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { private var count = 0 // The reverse mapping of _dictionary, i.e. mapping encoded integer to the value itself. - private var values = new mutable.ArrayBuffer[Any](1024) + private var values = new mutable.ArrayBuffer[T#JvmType](1024) // The dictionary that maps a value to the encoded short integer. private val dictionary = mutable.HashMap.empty[Any, Short] @@ -210,10 +199,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { // to store dictionary element count. private var dictionarySize = 4 - override def gatherCompressibilityStats[T <: NativeType]( - value: T#JvmType, - columnType: ColumnType[T, T#JvmType]) { - + override def gatherCompressibilityStats(value: T#JvmType, columnType: NativeColumnType[T]) { if (!overflow) { val actualSize = columnType.actualSize(value) count += 1 @@ -234,11 +220,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { } } - override def compress[T <: NativeType]( - from: ByteBuffer, - to: ByteBuffer, - columnType: ColumnType[T, T#JvmType]) = { - + override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]) = { if (overflow) { throw new IllegalStateException( "Dictionary encoding should not be used because of dictionary overflow.") @@ -249,7 +231,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { var i = 0 while (i < values.length) { - columnType.append(values(i).asInstanceOf[T#JvmType], to) + columnType.append(values(i), to) i += 1 } @@ -286,3 +268,215 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { override def hasNext = buffer.hasRemaining } } + +private[sql] case object BooleanBitSet extends CompressionScheme { + override val typeId = 3 + + val BITS_PER_LONG = 64 + + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + new this.Decoder(buffer).asInstanceOf[compression.Decoder[T]] + } + + override def encoder[T <: NativeType] = (new this.Encoder).asInstanceOf[compression.Encoder[T]] + + override def supports(columnType: ColumnType[_, _]) = columnType == BOOLEAN + + class Encoder extends compression.Encoder[BooleanType.type] { + private var _uncompressedSize = 0 + + override def gatherCompressibilityStats( + value: Boolean, + columnType: NativeColumnType[BooleanType.type]) { + + _uncompressedSize += BOOLEAN.defaultSize + } + + override def compress( + from: ByteBuffer, + to: ByteBuffer, + columnType: NativeColumnType[BooleanType.type]) = { + + to.putInt(BooleanBitSet.typeId) + // Total element count (1 byte per Boolean value) + .putInt(from.remaining) + + while (from.remaining >= BITS_PER_LONG) { + var word = 0: Long + var i = 0 + + while (i < BITS_PER_LONG) { + if (BOOLEAN.extract(from)) { + word |= (1: Long) << i + } + i += 1 + } + + to.putLong(word) + } + + if (from.hasRemaining) { + var word = 0: Long + var i = 0 + + while (from.hasRemaining) { + if (BOOLEAN.extract(from)) { + word |= (1: Long) << i + } + i += 1 + } + + to.putLong(word) + } + + to.rewind() + to + } + + override def uncompressedSize = _uncompressedSize + + override def compressedSize = { + val extra = if (_uncompressedSize % BITS_PER_LONG == 0) 0 else 1 + (_uncompressedSize / BITS_PER_LONG + extra) * 8 + 4 + } + } + + class Decoder(buffer: ByteBuffer) extends compression.Decoder[BooleanType.type] { + private val count = buffer.getInt() + + private var currentWord = 0: Long + + private var visited: Int = 0 + + override def next(): Boolean = { + val bit = visited % BITS_PER_LONG + + visited += 1 + if (bit == 0) { + currentWord = buffer.getLong() + } + + ((currentWord >> bit) & 1) != 0 + } + + override def hasNext: Boolean = visited < count + } +} + +private[sql] sealed abstract class IntegralDelta[I <: IntegralType] extends CompressionScheme { + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + new this.Decoder(buffer, columnType.asInstanceOf[NativeColumnType[I]]) + .asInstanceOf[compression.Decoder[T]] + } + + override def encoder[T <: NativeType] = (new this.Encoder).asInstanceOf[compression.Encoder[T]] + + /** + * Computes `delta = x - y`, returns `(true, delta)` if `delta` can fit into a single byte, or + * `(false, 0: Byte)` otherwise. + */ + protected def byteSizedDelta(x: I#JvmType, y: I#JvmType): (Boolean, Byte) + + /** + * Simply computes `x + delta` + */ + protected def addDelta(x: I#JvmType, delta: Byte): I#JvmType + + class Encoder extends compression.Encoder[I] { + private var _compressedSize: Int = 0 + + private var _uncompressedSize: Int = 0 + + private var prev: I#JvmType = _ + + private var initial = true + + override def gatherCompressibilityStats(value: I#JvmType, columnType: NativeColumnType[I]) { + _uncompressedSize += columnType.defaultSize + + if (initial) { + initial = false + prev = value + _compressedSize += 1 + columnType.defaultSize + } else { + val (smallEnough, _) = byteSizedDelta(value, prev) + _compressedSize += (if (smallEnough) 1 else 1 + columnType.defaultSize) + } + } + + override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[I]) = { + to.putInt(typeId) + + if (from.hasRemaining) { + val prev = columnType.extract(from) + + to.put(Byte.MinValue) + columnType.append(prev, to) + + while (from.hasRemaining) { + val current = columnType.extract(from) + val (smallEnough, delta) = byteSizedDelta(current, prev) + + if (smallEnough) { + to.put(delta) + } else { + to.put(Byte.MinValue) + columnType.append(current, to) + } + } + } + + to.rewind() + to + } + + override def uncompressedSize = _uncompressedSize + + override def compressedSize = _compressedSize + } + + class Decoder(buffer: ByteBuffer, columnType: NativeColumnType[I]) + extends compression.Decoder[I] { + + private var prev: I#JvmType = _ + + override def next() = { + val delta = buffer.get() + + if (delta > Byte.MinValue) { + addDelta(prev, delta) + } else { + prev = columnType.extract(buffer) + prev + } + } + + override def hasNext = buffer.hasRemaining + } +} + +private[sql] case object IntDelta extends IntegralDelta[IntegerType.type] { + override val typeId = 4 + + override def supports(columnType: ColumnType[_, _]) = columnType == INT + + override protected def addDelta(x: Int, delta: Byte) = x + delta + + override protected def byteSizedDelta(x: Int, y: Int): (Boolean, Byte) = { + val delta = x - y + if (delta < Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) + } +} + +private[sql] case object LongDelta extends IntegralDelta[LongType.type] { + override val typeId = 5 + + override def supports(columnType: ColumnType[_, _]) = columnType == LONG + + override protected def addDelta(x: Long, delta: Byte) = x + delta + + override protected def byteSizedDelta(x: Long, y: Long): (Boolean, Byte) = { + val delta = x - y + if (delta < Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala index 70b2e851737f8..2ed4cf2170f9d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala @@ -31,4 +31,12 @@ class ColumnarQuerySuite extends QueryTest { checkAnswer(scan, testData.collect().toSeq) } + + test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { + val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan + val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan)) + + checkAnswer(scan, testData.collect().toSeq) + checkAnswer(scan, testData.collect().toSeq) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala new file mode 100644 index 0000000000000..a754f98f7fbf1 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar.compression + +import org.scalatest.FunSuite + +import org.apache.spark.sql.Row +import org.apache.spark.sql.columnar.{BOOLEAN, BooleanColumnStats} +import org.apache.spark.sql.columnar.ColumnarTestUtils._ + +class BooleanBitSetSuite extends FunSuite { + import BooleanBitSet._ + + def skeleton(count: Int) { + // ------------- + // Tests encoder + // ------------- + + val builder = TestCompressibleColumnBuilder(new BooleanColumnStats, BOOLEAN, BooleanBitSet) + val rows = Seq.fill[Row](count)(makeRandomRow(BOOLEAN)) + val values = rows.map(_.head) + + rows.foreach(builder.appendFrom(_, 0)) + val buffer = builder.build() + + // Column type ID + null count + null positions + val headerSize = CompressionScheme.columnHeaderSize(buffer) + + // Compression scheme ID + element count + bitset words + val compressedSize = 4 + 4 + { + val extra = if (count % BITS_PER_LONG == 0) 0 else 1 + (count / BITS_PER_LONG + extra) * 8 + } + + // 4 extra bytes for compression scheme type ID + expectResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) + + // Skips column header + buffer.position(headerSize) + expectResult(BooleanBitSet.typeId, "Wrong compression scheme ID")(buffer.getInt()) + expectResult(count, "Wrong element count")(buffer.getInt()) + + var word = 0: Long + for (i <- 0 until count) { + val bit = i % BITS_PER_LONG + word = if (bit == 0) buffer.getLong() else word + expectResult(values(i), s"Wrong value in compressed buffer, index=$i") { + (word & ((1: Long) << bit)) != 0 + } + } + + // ------------- + // Tests decoder + // ------------- + + // Rewinds, skips column header and 4 more bytes for compression scheme ID + buffer.rewind().position(headerSize + 4) + + val decoder = BooleanBitSet.decoder(buffer, BOOLEAN) + values.foreach(expectResult(_, "Wrong decoded value")(decoder.next())) + assert(!decoder.hasNext) + } + + test(s"$BooleanBitSet: empty") { + skeleton(0) + } + + test(s"$BooleanBitSet: less than 1 word") { + skeleton(BITS_PER_LONG - 1) + } + + test(s"$BooleanBitSet: exactly 1 word") { + skeleton(BITS_PER_LONG) + } + + test(s"$BooleanBitSet: multiple whole words") { + skeleton(BITS_PER_LONG * 2) + } + + test(s"$BooleanBitSet: multiple words and 1 more bit") { + skeleton(BITS_PER_LONG * 2 + 1) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala index 184691ab5b46a..eab27987e08ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -24,7 +24,6 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow class DictionaryEncodingSuite extends FunSuite { testDictionaryEncoding(new IntColumnStats, INT) @@ -41,73 +40,82 @@ class DictionaryEncodingSuite extends FunSuite { (0 until buffer.getInt()).map(columnType.extract(buffer) -> _.toShort).toMap } - test(s"$DictionaryEncoding with $typeName: simple case") { + def stableDistinct(seq: Seq[Int]): Seq[Int] = if (seq.isEmpty) { + Seq.empty + } else { + seq.head +: seq.tail.filterNot(_ == seq.head) + } + + def skeleton(uniqueValueCount: Int, inputSeq: Seq[Int]) { // ------------- // Tests encoder // ------------- val builder = TestCompressibleColumnBuilder(columnStats, columnType, DictionaryEncoding) - val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, 2) - - builder.initialize(0) - builder.appendFrom(rows(0), 0) - builder.appendFrom(rows(1), 0) - builder.appendFrom(rows(0), 0) - builder.appendFrom(rows(1), 0) - - val buffer = builder.build() - val headerSize = CompressionScheme.columnHeaderSize(buffer) - // 4 extra bytes for dictionary size - val dictionarySize = 4 + values.map(columnType.actualSize).sum - // 4 `Short`s, 2 bytes each - val compressedSize = dictionarySize + 2 * 4 - // 4 extra bytes for compression scheme type ID - expectResult(headerSize + 4 + compressedSize, "Wrong buffer capacity")(buffer.capacity) - - // Skips column header - buffer.position(headerSize) - expectResult(DictionaryEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) - - val dictionary = buildDictionary(buffer) - Array[Short](0, 1).foreach { i => - expectResult(i, "Wrong dictionary entry")(dictionary(values(i))) - } - - Array[Short](0, 1, 0, 1).foreach { - expectResult(_, "Wrong column element value")(buffer.getShort()) + val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, uniqueValueCount) + val dictValues = stableDistinct(inputSeq) + + inputSeq.foreach(i => builder.appendFrom(rows(i), 0)) + + if (dictValues.length > DictionaryEncoding.MAX_DICT_SIZE) { + withClue("Dictionary overflowed, compression should fail") { + intercept[Throwable] { + builder.build() + } + } + } else { + val buffer = builder.build() + val headerSize = CompressionScheme.columnHeaderSize(buffer) + // 4 extra bytes for dictionary size + val dictionarySize = 4 + values.map(columnType.actualSize).sum + // 2 bytes for each `Short` + val compressedSize = 4 + dictionarySize + 2 * inputSeq.length + // 4 extra bytes for compression scheme type ID + expectResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) + + // Skips column header + buffer.position(headerSize) + expectResult(DictionaryEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) + + val dictionary = buildDictionary(buffer).toMap + + dictValues.foreach { i => + expectResult(i, "Wrong dictionary entry") { + dictionary(values(i)) + } + } + + inputSeq.foreach { i => + expectResult(i.toShort, "Wrong column element value")(buffer.getShort()) + } + + // ------------- + // Tests decoder + // ------------- + + // Rewinds, skips column header and 4 more bytes for compression scheme ID + buffer.rewind().position(headerSize + 4) + + val decoder = DictionaryEncoding.decoder(buffer, columnType) + + inputSeq.foreach { i => + expectResult(values(i), "Wrong decoded value")(decoder.next()) + } + + assert(!decoder.hasNext) } - - // ------------- - // Tests decoder - // ------------- - - // Rewinds, skips column header and 4 more bytes for compression scheme ID - buffer.rewind().position(headerSize + 4) - - val decoder = new DictionaryEncoding.Decoder[T](buffer, columnType) - - Array[Short](0, 1, 0, 1).foreach { i => - expectResult(values(i), "Wrong decoded value")(decoder.next()) - } - - assert(!decoder.hasNext) } - } - test(s"$DictionaryEncoding: overflow") { - val builder = TestCompressibleColumnBuilder(new IntColumnStats, INT, DictionaryEncoding) - builder.initialize(0) + test(s"$DictionaryEncoding with $typeName: empty") { + skeleton(0, Seq.empty) + } - (0 to Short.MaxValue).foreach { n => - val row = new GenericMutableRow(1) - row.setInt(0, n) - builder.appendFrom(row, 0) + test(s"$DictionaryEncoding with $typeName: simple case") { + skeleton(2, Seq(0, 1, 0, 1)) } - withClue("Dictionary overflowed, encoding should fail") { - intercept[Throwable] { - builder.build() - } + test(s"$DictionaryEncoding with $typeName: dictionary overflow") { + skeleton(DictionaryEncoding.MAX_DICT_SIZE + 1, 0 to DictionaryEncoding.MAX_DICT_SIZE) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala new file mode 100644 index 0000000000000..1390e5eef6106 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar.compression + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.types.IntegralType +import org.apache.spark.sql.columnar._ + +class IntegralDeltaSuite extends FunSuite { + testIntegralDelta(new IntColumnStats, INT, IntDelta) + testIntegralDelta(new LongColumnStats, LONG, LongDelta) + + def testIntegralDelta[I <: IntegralType]( + columnStats: NativeColumnStats[I], + columnType: NativeColumnType[I], + scheme: IntegralDelta[I]) { + + def skeleton(input: Seq[I#JvmType]) { + // ------------- + // Tests encoder + // ------------- + + val builder = TestCompressibleColumnBuilder(columnStats, columnType, scheme) + val deltas = if (input.isEmpty) { + Seq.empty[Long] + } else { + (input.tail, input.init).zipped.map { + case (x: Int, y: Int) => (x - y).toLong + case (x: Long, y: Long) => x - y + } + } + + input.map { value => + val row = new GenericMutableRow(1) + columnType.setField(row, 0, value) + builder.appendFrom(row, 0) + } + + val buffer = builder.build() + // Column type ID + null count + null positions + val headerSize = CompressionScheme.columnHeaderSize(buffer) + + // Compression scheme ID + compressed contents + val compressedSize = 4 + (if (deltas.isEmpty) { + 0 + } else { + val oneBoolean = columnType.defaultSize + 1 + oneBoolean + deltas.map { + d => if (math.abs(d) < Byte.MaxValue) 1 else 1 + oneBoolean + }.sum + }) + + // 4 extra bytes for compression scheme type ID + expectResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) + + buffer.position(headerSize) + expectResult(scheme.typeId, "Wrong compression scheme ID")(buffer.getInt()) + + if (input.nonEmpty) { + expectResult(Byte.MinValue, "The first byte should be an escaping mark")(buffer.get()) + expectResult(input.head, "The first value is wrong")(columnType.extract(buffer)) + + (input.tail, deltas).zipped.foreach { (value, delta) => + if (delta < Byte.MaxValue) { + expectResult(delta, "Wrong delta")(buffer.get()) + } else { + expectResult(Byte.MinValue, "Expecting escaping mark here")(buffer.get()) + expectResult(value, "Wrong value")(columnType.extract(buffer)) + } + } + } + + // ------------- + // Tests decoder + // ------------- + + // Rewinds, skips column header and 4 more bytes for compression scheme ID + buffer.rewind().position(headerSize + 4) + + val decoder = scheme.decoder(buffer, columnType) + input.foreach(expectResult(_, "Wrong decoded value")(decoder.next())) + assert(!decoder.hasNext) + } + + test(s"$scheme: empty column") { + skeleton(Seq.empty) + } + + test(s"$scheme: simple case") { + val input = columnType match { + case INT => Seq(1: Int, 2: Int, 130: Int) + case LONG => Seq(1: Long, 2: Long, 130: Long) + } + + skeleton(input.map(_.asInstanceOf[I#JvmType])) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala index 2089ad120d4f2..89f9b60a4397b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -37,34 +37,39 @@ class RunLengthEncodingSuite extends FunSuite { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") - test(s"$RunLengthEncoding with $typeName: simple case") { + def skeleton(uniqueValueCount: Int, inputRuns: Seq[(Int, Int)]) { // ------------- // Tests encoder // ------------- val builder = TestCompressibleColumnBuilder(columnStats, columnType, RunLengthEncoding) - val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, 2) - - builder.initialize(0) - builder.appendFrom(rows(0), 0) - builder.appendFrom(rows(0), 0) - builder.appendFrom(rows(1), 0) - builder.appendFrom(rows(1), 0) + val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, uniqueValueCount) + val inputSeq = inputRuns.flatMap { case (index, run) => + Seq.fill(run)(index) + } + inputSeq.foreach(i => builder.appendFrom(rows(i), 0)) val buffer = builder.build() + + // Column type ID + null count + null positions val headerSize = CompressionScheme.columnHeaderSize(buffer) - // 4 extra bytes each run for run length - val compressedSize = values.map(columnType.actualSize(_) + 4).sum + + // Compression scheme ID + compressed contents + val compressedSize = 4 + inputRuns.map { case (index, _) => + // 4 extra bytes each run for run length + columnType.actualSize(values(index)) + 4 + }.sum + // 4 extra bytes for compression scheme type ID - expectResult(headerSize + 4 + compressedSize, "Wrong buffer capacity")(buffer.capacity) + expectResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) // Skips column header buffer.position(headerSize) expectResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) - Array(0, 1).foreach { i => - expectResult(values(i), "Wrong column element value")(columnType.extract(buffer)) - expectResult(2, "Wrong run length")(buffer.getInt()) + inputRuns.foreach { case (index, run) => + expectResult(values(index), "Wrong column element value")(columnType.extract(buffer)) + expectResult(run, "Wrong run length")(buffer.getInt()) } // ------------- @@ -74,57 +79,29 @@ class RunLengthEncodingSuite extends FunSuite { // Rewinds, skips column header and 4 more bytes for compression scheme ID buffer.rewind().position(headerSize + 4) - val decoder = new RunLengthEncoding.Decoder[T](buffer, columnType) + val decoder = RunLengthEncoding.decoder(buffer, columnType) - Array(0, 0, 1, 1).foreach { i => + inputSeq.foreach { i => expectResult(values(i), "Wrong decoded value")(decoder.next()) } assert(!decoder.hasNext) } - test(s"$RunLengthEncoding with $typeName: run length == 1") { - // ------------- - // Tests encoder - // ------------- - - val builder = TestCompressibleColumnBuilder(columnStats, columnType, RunLengthEncoding) - val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, 2) - - builder.initialize(0) - builder.appendFrom(rows(0), 0) - builder.appendFrom(rows(1), 0) - - val buffer = builder.build() - val headerSize = CompressionScheme.columnHeaderSize(buffer) - // 4 bytes each run for run length - val compressedSize = values.map(columnType.actualSize(_) + 4).sum - // 4 bytes for compression scheme type ID - expectResult(headerSize + 4 + compressedSize, "Wrong buffer capacity")(buffer.capacity) - - // Skips column header - buffer.position(headerSize) - expectResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) - - Array(0, 1).foreach { i => - expectResult(values(i), "Wrong column element value")(columnType.extract(buffer)) - expectResult(1, "Wrong run length")(buffer.getInt()) - } - - // ------------- - // Tests decoder - // ------------- - - // Rewinds, skips column header and 4 more bytes for compression scheme ID - buffer.rewind().position(headerSize + 4) + test(s"$RunLengthEncoding with $typeName: empty column") { + skeleton(0, Seq.empty) + } - val decoder = new RunLengthEncoding.Decoder[T](buffer, columnType) + test(s"$RunLengthEncoding with $typeName: simple case") { + skeleton(2, Seq(0 -> 2, 1 ->2)) + } - Array(0, 1).foreach { i => - expectResult(values(i), "Wrong decoded value")(decoder.next()) - } + test(s"$RunLengthEncoding with $typeName: run length == 1") { + skeleton(2, Seq(0 -> 1, 1 ->1)) + } - assert(!decoder.hasNext) + test(s"$RunLengthEncoding with $typeName: single long run") { + skeleton(1, Seq(0 -> 1000)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala index e0ec812863dcf..81bf5e99d19b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala @@ -28,7 +28,7 @@ class TestCompressibleColumnBuilder[T <: NativeType]( with NullableColumnBuilder with CompressibleColumnBuilder[T] { - override protected def isWorthCompressing(encoder: Encoder) = true + override protected def isWorthCompressing(encoder: Encoder[T]) = true } object TestCompressibleColumnBuilder { @@ -37,7 +37,9 @@ object TestCompressibleColumnBuilder { columnType: NativeColumnType[T], scheme: CompressionScheme) = { - new TestCompressibleColumnBuilder(columnStats, columnType, Seq(scheme)) + val builder = new TestCompressibleColumnBuilder(columnStats, columnType, Seq(scheme)) + builder.initialize(0) + builder } } From 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 7 Apr 2014 23:40:21 -0700 Subject: [PATCH 024/641] [SPARK-1103] Automatic garbage collection of RDD, shuffle and broadcast data MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR allows Spark to automatically cleanup metadata and data related to persisted RDDs, shuffles and broadcast variables when the corresponding RDDs, shuffles and broadcast variables fall out of scope from the driver program. This is still a work in progress as broadcast cleanup has not been implemented. **Implementation Details** A new class `ContextCleaner` is responsible cleaning all the state. It is instantiated as part of a `SparkContext`. RDD and ShuffleDependency classes have overridden `finalize()` function that gets called whenever their instances go out of scope. The `finalize()` function enqueues the object’s identifier (i.e. RDD ID, shuffle ID, etc.) with the `ContextCleaner`, which is a very short and cheap operation and should not significantly affect the garbage collection mechanism. The `ContextCleaner`, on a different thread, performs the cleanup, whose details are given below. *RDD cleanup:* `ContextCleaner` calls `RDD.unpersist()` is used to cleanup persisted RDDs. Regarding metadata, the DAGScheduler automatically cleans up all metadata related to a RDD after all jobs have completed. Only the `SparkContext.persistentRDDs` keeps strong references to persisted RDDs. The `TimeStampedHashMap` used for that has been replaced by `TimeStampedWeakValueHashMap` that keeps only weak references to the RDDs, allowing them to be garbage collected. *Shuffle cleanup:* New BlockManager message `RemoveShuffle()` asks the `BlockManagerMaster` and currently active `BlockManager`s to delete all the disk blocks related to the shuffle ID. `ContextCleaner` cleans up shuffle data using this message and also cleans up the metadata in the `MapOutputTracker` of the driver. The `MapOutputTracker` at the workers, that caches the shuffle metadata, maintains a `BoundedHashMap` to limit the shuffle information it caches. Refetching the shuffle information from the driver is not too costly. *Broadcast cleanup:* To be done. [This PR](https://github.com/apache/incubator-spark/pull/543/) adds mechanism for explicit cleanup of broadcast variables. `Broadcast.finalize()` will enqueue its own ID with ContextCleaner and the PRs mechanism will be used to unpersist the Broadcast data. *Other cleanup:* `ShuffleMapTask` and `ResultTask` caches tasks and used TTL based cleanup (using `TimeStampedHashMap`), so nothing got cleaned up if TTL was not set. Instead, they now use `BoundedHashMap` to keep a limited number of map output information. Cost of repopulating the cache if necessary is very small. **Current state of implementation** Implemented RDD and shuffle cleanup. Things left to be done are. - Cleaning up for broadcast variable still to be done. - Automatic cleaning up keys with empty weak refs as values in `TimeStampedWeakValueHashMap` Author: Tathagata Das Author: Andrew Or Author: Roman Pastukhov Closes #126 from tdas/state-cleanup and squashes the following commits: 61b8d6e [Tathagata Das] Fixed issue with Tachyon + new BlockManager methods. f489fdc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup d25a86e [Tathagata Das] Fixed stupid typo. cff023c [Tathagata Das] Fixed issues based on Andrew's comments. 4d05314 [Tathagata Das] Scala style fix. 2b95b5e [Tathagata Das] Added more documentation on Broadcast implementations, specially which blocks are told about to the driver. Also, fixed Broadcast API to hide destroy functionality. 41c9ece [Tathagata Das] Added more unit tests for BlockManager, DiskBlockManager, and ContextCleaner. 6222697 [Tathagata Das] Fixed bug and adding unit test for removeBroadcast in BlockManagerSuite. 104a89a [Tathagata Das] Fixed failing BroadcastSuite unit tests by introducing blocking for removeShuffle and removeBroadcast in BlockManager* a430f06 [Tathagata Das] Fixed compilation errors. b27f8e8 [Tathagata Das] Merge pull request #3 from andrewor14/cleanup cd72d19 [Andrew Or] Make automatic cleanup configurable (not documented) ada45f0 [Andrew Or] Merge branch 'state-cleanup' of github.com:tdas/spark into cleanup a2cc8bc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup c5b1d98 [Andrew Or] Address Patrick's comments a6460d4 [Andrew Or] Merge github.com:apache/spark into cleanup 762a4d8 [Tathagata Das] Merge pull request #1 from andrewor14/cleanup f0aabb1 [Andrew Or] Correct semantics for TimeStampedWeakValueHashMap + add tests 5016375 [Andrew Or] Address TD's comments 7ed72fb [Andrew Or] Fix style test fail + remove verbose test message regarding broadcast 634a097 [Andrew Or] Merge branch 'state-cleanup' of github.com:tdas/spark into cleanup 7edbc98 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into state-cleanup 8557c12 [Andrew Or] Merge github.com:apache/spark into cleanup e442246 [Andrew Or] Merge github.com:apache/spark into cleanup 88904a3 [Andrew Or] Make TimeStampedWeakValueHashMap a wrapper of TimeStampedHashMap fbfeec8 [Andrew Or] Add functionality to query executors for their local BlockStatuses 34f436f [Andrew Or] Generalize BroadcastBlockId to remove BroadcastHelperBlockId 0d17060 [Andrew Or] Import, comments, and style fixes (minor) c92e4d9 [Andrew Or] Merge github.com:apache/spark into cleanup f201a8d [Andrew Or] Test broadcast cleanup in ContextCleanerSuite + remove BoundedHashMap e95479c [Andrew Or] Add tests for unpersisting broadcast 544ac86 [Andrew Or] Clean up broadcast blocks through BlockManager* d0edef3 [Andrew Or] Add framework for broadcast cleanup ba52e00 [Andrew Or] Refactor broadcast classes c7ccef1 [Andrew Or] Merge branch 'bc-unpersist-merge' of github.com:ignatich/incubator-spark into cleanup 6c9dcf6 [Tathagata Das] Added missing Apache license d2f8b97 [Tathagata Das] Removed duplicate unpersistRDD. a007307 [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup 620eca3 [Tathagata Das] Changes based on PR comments. f2881fd [Tathagata Das] Changed ContextCleaner to use ReferenceQueue instead of finalizer e1fba5f [Tathagata Das] Style fix 892b952 [Tathagata Das] Removed use of BoundedHashMap, and made BlockManagerSlaveActor cleanup shuffle metadata in MapOutputTrackerWorker. a7260d3 [Tathagata Das] Added try-catch in context cleaner and null value cleaning in TimeStampedWeakValueHashMap. e61daa0 [Tathagata Das] Modifications based on the comments on PR 126. ae9da88 [Tathagata Das] Removed unncessary TimeStampedHashMap from DAGScheduler, added try-catches in finalize() methods, and replaced ArrayBlockingQueue to LinkedBlockingQueue to avoid blocking in Java's finalizing thread. cb0a5a6 [Tathagata Das] Fixed docs and styles. a24fefc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup 8512612 [Tathagata Das] Changed TimeStampedHashMap to use WrappedJavaHashMap. e427a9e [Tathagata Das] Added ContextCleaner to automatically clean RDDs and shuffles when they fall out of scope. Also replaced TimeStampedHashMap to BoundedHashMaps and TimeStampedWeakValueHashMap for the necessary hashmap behavior. 80dd977 [Roman Pastukhov] Fix for Broadcast unpersist patch. 1e752f1 [Roman Pastukhov] Added unpersist method to Broadcast. --- .../org/apache/spark/ContextCleaner.scala | 192 ++++++++ .../scala/org/apache/spark/Dependency.scala | 2 + .../org/apache/spark/MapOutputTracker.scala | 148 ++++--- .../scala/org/apache/spark/SparkContext.scala | 23 +- .../scala/org/apache/spark/SparkEnv.scala | 25 +- .../apache/spark/broadcast/Broadcast.scala | 107 +++-- .../spark/broadcast/BroadcastFactory.scala | 3 +- .../spark/broadcast/BroadcastManager.scala | 66 +++ .../spark/broadcast/HttpBroadcast.scala | 128 ++++-- .../broadcast/HttpBroadcastFactory.scala | 45 ++ .../spark/broadcast/TorrentBroadcast.scala | 162 ++++--- .../broadcast/TorrentBroadcastFactory.scala | 46 ++ .../spark/network/ConnectionManager.scala | 1 - .../main/scala/org/apache/spark/rdd/RDD.scala | 5 +- .../apache/spark/scheduler/DAGScheduler.scala | 38 +- .../apache/spark/scheduler/ResultTask.scala | 16 +- .../spark/scheduler/ShuffleMapTask.scala | 14 +- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../org/apache/spark/storage/BlockId.scala | 24 +- .../apache/spark/storage/BlockManager.scala | 67 ++- .../spark/storage/BlockManagerMaster.scala | 84 +++- .../storage/BlockManagerMasterActor.scala | 107 ++++- .../spark/storage/BlockManagerMessages.scala | 20 +- .../storage/BlockManagerSlaveActor.scala | 60 ++- .../spark/storage/DiskBlockManager.scala | 14 + .../spark/storage/ShuffleBlockManager.scala | 44 +- .../apache/spark/storage/ThreadingTest.scala | 6 +- .../apache/spark/util/MetadataCleaner.scala | 19 +- .../spark/util/TimeStampedHashMap.scala | 109 ++--- .../util/TimeStampedWeakValueHashMap.scala | 170 +++++++ .../scala/org/apache/spark/util/Utils.scala | 8 +- .../org/apache/spark/AkkaUtilsSuite.scala | 8 +- .../org/apache/spark/BroadcastSuite.scala | 311 +++++++++++-- .../apache/spark/ContextCleanerSuite.scala | 415 ++++++++++++++++++ .../apache/spark/MapOutputTrackerSuite.scala | 25 +- .../spark/storage/BlockManagerSuite.scala | 243 ++++++++-- .../spark/storage/DiskBlockManagerSuite.scala | 10 +- .../apache/spark/util/JsonProtocolSuite.scala | 5 +- .../spark/util/TimeStampedHashMapSuite.scala | 264 +++++++++++ .../spark/streaming/dstream/DStream.scala | 4 +- 40 files changed, 2571 insertions(+), 469 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/ContextCleaner.scala create mode 100644 core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala create mode 100644 core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala create mode 100644 core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala create mode 100644 core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala create mode 100644 core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala new file mode 100644 index 0000000000000..54e08d7866f75 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -0,0 +1,192 @@ +/* + * 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 + +import java.lang.ref.{ReferenceQueue, WeakReference} + +import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD + +/** + * Classes that represent cleaning tasks. + */ +private sealed trait CleanupTask +private case class CleanRDD(rddId: Int) extends CleanupTask +private case class CleanShuffle(shuffleId: Int) extends CleanupTask +private case class CleanBroadcast(broadcastId: Long) extends CleanupTask + +/** + * A WeakReference associated with a CleanupTask. + * + * When the referent object becomes only weakly reachable, the corresponding + * CleanupTaskWeakReference is automatically added to the given reference queue. + */ +private class CleanupTaskWeakReference( + val task: CleanupTask, + referent: AnyRef, + referenceQueue: ReferenceQueue[AnyRef]) + extends WeakReference(referent, referenceQueue) + +/** + * An asynchronous cleaner for RDD, shuffle, and broadcast state. + * + * This maintains a weak reference for each RDD, ShuffleDependency, and Broadcast of interest, + * to be processed when the associated object goes out of scope of the application. Actual + * cleanup is performed in a separate daemon thread. + */ +private[spark] class ContextCleaner(sc: SparkContext) extends Logging { + + private val referenceBuffer = new ArrayBuffer[CleanupTaskWeakReference] + with SynchronizedBuffer[CleanupTaskWeakReference] + + private val referenceQueue = new ReferenceQueue[AnyRef] + + private val listeners = new ArrayBuffer[CleanerListener] + with SynchronizedBuffer[CleanerListener] + + private val cleaningThread = new Thread() { override def run() { keepCleaning() }} + + /** + * Whether the cleaning thread will block on cleanup tasks. + * This is set to true only for tests. + */ + private val blockOnCleanupTasks = sc.conf.getBoolean( + "spark.cleaner.referenceTracking.blocking", false) + + @volatile private var stopped = false + + /** Attach a listener object to get information of when objects are cleaned. */ + def attachListener(listener: CleanerListener) { + listeners += listener + } + + /** Start the cleaner. */ + def start() { + cleaningThread.setDaemon(true) + cleaningThread.setName("Spark Context Cleaner") + cleaningThread.start() + } + + /** Stop the cleaner. */ + def stop() { + stopped = true + } + + /** Register a RDD for cleanup when it is garbage collected. */ + def registerRDDForCleanup(rdd: RDD[_]) { + registerForCleanup(rdd, CleanRDD(rdd.id)) + } + + /** Register a ShuffleDependency for cleanup when it is garbage collected. */ + def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _]) { + registerForCleanup(shuffleDependency, CleanShuffle(shuffleDependency.shuffleId)) + } + + /** Register a Broadcast for cleanup when it is garbage collected. */ + def registerBroadcastForCleanup[T](broadcast: Broadcast[T]) { + registerForCleanup(broadcast, CleanBroadcast(broadcast.id)) + } + + /** Register an object for cleanup. */ + private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask) { + referenceBuffer += new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue) + } + + /** Keep cleaning RDD, shuffle, and broadcast state. */ + private def keepCleaning() { + while (!stopped) { + try { + val reference = Option(referenceQueue.remove(ContextCleaner.REF_QUEUE_POLL_TIMEOUT)) + .map(_.asInstanceOf[CleanupTaskWeakReference]) + reference.map(_.task).foreach { task => + logDebug("Got cleaning task " + task) + referenceBuffer -= reference.get + task match { + case CleanRDD(rddId) => + doCleanupRDD(rddId, blocking = blockOnCleanupTasks) + case CleanShuffle(shuffleId) => + doCleanupShuffle(shuffleId, blocking = blockOnCleanupTasks) + case CleanBroadcast(broadcastId) => + doCleanupBroadcast(broadcastId, blocking = blockOnCleanupTasks) + } + } + } catch { + case t: Throwable => logError("Error in cleaning thread", t) + } + } + } + + /** Perform RDD cleanup. */ + def doCleanupRDD(rddId: Int, blocking: Boolean) { + try { + logDebug("Cleaning RDD " + rddId) + sc.unpersistRDD(rddId, blocking) + listeners.foreach(_.rddCleaned(rddId)) + logInfo("Cleaned RDD " + rddId) + } catch { + case t: Throwable => logError("Error cleaning RDD " + rddId, t) + } + } + + /** Perform shuffle cleanup, asynchronously. */ + def doCleanupShuffle(shuffleId: Int, blocking: Boolean) { + try { + logDebug("Cleaning shuffle " + shuffleId) + mapOutputTrackerMaster.unregisterShuffle(shuffleId) + blockManagerMaster.removeShuffle(shuffleId, blocking) + listeners.foreach(_.shuffleCleaned(shuffleId)) + logInfo("Cleaned shuffle " + shuffleId) + } catch { + case t: Throwable => logError("Error cleaning shuffle " + shuffleId, t) + } + } + + /** Perform broadcast cleanup. */ + def doCleanupBroadcast(broadcastId: Long, blocking: Boolean) { + try { + logDebug("Cleaning broadcast " + broadcastId) + broadcastManager.unbroadcast(broadcastId, true, blocking) + listeners.foreach(_.broadcastCleaned(broadcastId)) + logInfo("Cleaned broadcast " + broadcastId) + } catch { + case t: Throwable => logError("Error cleaning broadcast " + broadcastId, t) + } + } + + private def blockManagerMaster = sc.env.blockManager.master + private def broadcastManager = sc.env.broadcastManager + private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + + // Used for testing. These methods explicitly blocks until cleanup is completed + // to ensure that more reliable testing. +} + +private object ContextCleaner { + private val REF_QUEUE_POLL_TIMEOUT = 100 +} + +/** + * Listener class used for testing when any item has been cleaned by the Cleaner class. + */ +private[spark] trait CleanerListener { + def rddCleaned(rddId: Int) + def shuffleCleaned(shuffleId: Int) + def broadcastCleaned(broadcastId: Long) +} diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 3132dcf745e19..1cd629c15bd46 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -55,6 +55,8 @@ class ShuffleDependency[K, V]( extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) { val shuffleId: Int = rdd.context.newShuffleId() + + rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 80cbf951cb70e..ee82d9fa7874b 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -20,21 +20,21 @@ package org.apache.spark import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} -import scala.collection.mutable.HashSet +import scala.collection.mutable.{HashSet, HashMap, Map} import scala.concurrent.Await import akka.actor._ import akka.pattern.ask - import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{AkkaUtils, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} +import org.apache.spark.util._ private[spark] sealed trait MapOutputTrackerMessage private[spark] case class GetMapOutputStatuses(shuffleId: Int) extends MapOutputTrackerMessage private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage +/** Actor class for MapOutputTrackerMaster */ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster, conf: SparkConf) extends Actor with Logging { val maxAkkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) @@ -65,26 +65,41 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster } } -private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { - +/** + * Class that keeps track of the location of the map output of + * a stage. This is abstract because different versions of MapOutputTracker + * (driver and worker) use different HashMap to store its metadata. + */ +private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging { private val timeout = AkkaUtils.askTimeout(conf) - // Set to the MapOutputTrackerActor living on the driver + /** Set to the MapOutputTrackerActor living on the driver. */ var trackerActor: ActorRef = _ - protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]] + /** + * This HashMap has different behavior for the master and the workers. + * + * On the master, it serves as the source of map outputs recorded from ShuffleMapTasks. + * On the workers, it simply serves as a cache, in which a miss triggers a fetch from the + * master's corresponding HashMap. + */ + protected val mapStatuses: Map[Int, Array[MapStatus]] - // Incremented every time a fetch fails so that client nodes know to clear - // their cache of map output locations if this happens. + /** + * Incremented every time a fetch fails so that client nodes know to clear + * their cache of map output locations if this happens. + */ protected var epoch: Long = 0 - protected val epochLock = new java.lang.Object + protected val epochLock = new AnyRef - private val metadataCleaner = - new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup, conf) + /** Remembers which map output locations are currently being fetched on a worker. */ + private val fetching = new HashSet[Int] - // Send a message to the trackerActor and get its result within a default timeout, or - // throw a SparkException if this fails. - private def askTracker(message: Any): Any = { + /** + * Send a message to the trackerActor and get its result within a default timeout, or + * throw a SparkException if this fails. + */ + protected def askTracker(message: Any): Any = { try { val future = trackerActor.ask(message)(timeout) Await.result(future, timeout) @@ -94,17 +109,17 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { } } - // Send a one-way message to the trackerActor, to which we expect it to reply with true. - private def communicate(message: Any) { + /** Send a one-way message to the trackerActor, to which we expect it to reply with true. */ + protected def sendTracker(message: Any) { if (askTracker(message) != true) { throw new SparkException("Error reply received from MapOutputTracker") } } - // Remembers which map output locations are currently being fetched on a worker - private val fetching = new HashSet[Int] - - // Called on possibly remote nodes to get the server URIs and output sizes for a given shuffle + /** + * Called from executors to get the server URIs and output sizes of the map outputs of + * a given shuffle. + */ def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Long)] = { val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { @@ -152,8 +167,7 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { fetchedStatuses.synchronized { return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, fetchedStatuses) } - } - else { + } else { throw new FetchFailedException(null, shuffleId, -1, reduceId, new Exception("Missing all output locations for shuffle " + shuffleId)) } @@ -164,27 +178,18 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { } } - protected def cleanup(cleanupTime: Long) { - mapStatuses.clearOldValues(cleanupTime) - } - - def stop() { - communicate(StopMapOutputTracker) - mapStatuses.clear() - metadataCleaner.cancel() - trackerActor = null - } - - // Called to get current epoch number + /** Called to get current epoch number. */ def getEpoch: Long = { epochLock.synchronized { return epoch } } - // Called on workers to update the epoch number, potentially clearing old outputs - // because of a fetch failure. (Each worker task calls this with the latest epoch - // number on the master at the time it was created.) + /** + * Called from executors to update the epoch number, potentially clearing old outputs + * because of a fetch failure. Each worker task calls this with the latest epoch + * number on the master at the time it was created. + */ def updateEpoch(newEpoch: Long) { epochLock.synchronized { if (newEpoch > epoch) { @@ -194,17 +199,40 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { } } } + + /** Unregister shuffle data. */ + def unregisterShuffle(shuffleId: Int) { + mapStatuses.remove(shuffleId) + } + + /** Stop the tracker. */ + def stop() { } } +/** + * MapOutputTracker for the driver. This uses TimeStampedHashMap to keep track of map + * output information, which allows old output information based on a TTL. + */ private[spark] class MapOutputTrackerMaster(conf: SparkConf) extends MapOutputTracker(conf) { - // Cache a serialized version of the output statuses for each shuffle to send them out faster + /** Cache a serialized version of the output statuses for each shuffle to send them out faster */ private var cacheEpoch = epoch - private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]] + + /** + * Timestamp based HashMap for storing mapStatuses and cached serialized statuses in the master, + * so that statuses are dropped only by explicit de-registering or by TTL-based cleaning (if set). + * Other than these two scenarios, nothing should be dropped from this HashMap. + */ + protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]() + private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]]() + + // For cleaning up TimeStampedHashMaps + private val metadataCleaner = + new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup, conf) def registerShuffle(shuffleId: Int, numMaps: Int) { - if (mapStatuses.putIfAbsent(shuffleId, new Array[MapStatus](numMaps)).isDefined) { + if (mapStatuses.put(shuffleId, new Array[MapStatus](numMaps)).isDefined) { throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice") } } @@ -216,6 +244,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } } + /** Register multiple map output information for the given shuffle */ def registerMapOutputs(shuffleId: Int, statuses: Array[MapStatus], changeEpoch: Boolean = false) { mapStatuses.put(shuffleId, Array[MapStatus]() ++ statuses) if (changeEpoch) { @@ -223,6 +252,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } } + /** Unregister map output information of the given shuffle, mapper and block manager */ def unregisterMapOutput(shuffleId: Int, mapId: Int, bmAddress: BlockManagerId) { val arrayOpt = mapStatuses.get(shuffleId) if (arrayOpt.isDefined && arrayOpt.get != null) { @@ -238,6 +268,17 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } } + /** Unregister shuffle data */ + override def unregisterShuffle(shuffleId: Int) { + mapStatuses.remove(shuffleId) + cachedSerializedStatuses.remove(shuffleId) + } + + /** Check if the given shuffle is being tracked */ + def containsShuffle(shuffleId: Int): Boolean = { + cachedSerializedStatuses.contains(shuffleId) || mapStatuses.contains(shuffleId) + } + def incrementEpoch() { epochLock.synchronized { epoch += 1 @@ -274,23 +315,26 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) bytes } - protected override def cleanup(cleanupTime: Long) { - super.cleanup(cleanupTime) - cachedSerializedStatuses.clearOldValues(cleanupTime) - } - override def stop() { - super.stop() + sendTracker(StopMapOutputTracker) + mapStatuses.clear() + trackerActor = null + metadataCleaner.cancel() cachedSerializedStatuses.clear() } - override def updateEpoch(newEpoch: Long) { - // This might be called on the MapOutputTrackerMaster if we're running in local mode. + private def cleanup(cleanupTime: Long) { + mapStatuses.clearOldValues(cleanupTime) + cachedSerializedStatuses.clearOldValues(cleanupTime) } +} - def has(shuffleId: Int): Boolean = { - cachedSerializedStatuses.get(shuffleId).isDefined || mapStatuses.contains(shuffleId) - } +/** + * MapOutputTracker for the workers, which fetches map output information from the driver's + * MapOutputTrackerMaster. + */ +private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTracker(conf) { + protected val mapStatuses = new HashMap[Int, Array[MapStatus]] } private[spark] object MapOutputTracker { diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e5ebd350eeced..d7124616d3bfb 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -45,7 +45,7 @@ import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, Me import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} +import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -157,7 +157,7 @@ class SparkContext( private[spark] val addedJars = HashMap[String, Long]() // Keeps track of all persisted RDDs - private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]] + private[spark] val persistentRdds = new TimeStampedWeakValueHashMap[Int, RDD[_]] private[spark] val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf) @@ -233,6 +233,15 @@ class SparkContext( @volatile private[spark] var dagScheduler = new DAGScheduler(this) dagScheduler.start() + private[spark] val cleaner: Option[ContextCleaner] = { + if (conf.getBoolean("spark.cleaner.referenceTracking", true)) { + Some(new ContextCleaner(this)) + } else { + None + } + } + cleaner.foreach(_.start()) + postEnvironmentUpdate() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ @@ -679,7 +688,11 @@ class SparkContext( * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. * The variable will be sent to each cluster only once. */ - def broadcast[T](value: T): Broadcast[T] = env.broadcastManager.newBroadcast[T](value, isLocal) + def broadcast[T](value: T): Broadcast[T] = { + val bc = env.broadcastManager.newBroadcast[T](value, isLocal) + cleaner.foreach(_.registerBroadcastForCleanup(bc)) + bc + } /** * Add a file to be downloaded with this Spark job on every node. @@ -789,8 +802,7 @@ class SparkContext( /** * Unpersist an RDD from memory and/or disk storage */ - private[spark] def unpersistRDD(rdd: RDD[_], blocking: Boolean = true) { - val rddId = rdd.id + private[spark] def unpersistRDD(rddId: Int, blocking: Boolean = true) { env.blockManager.master.removeRdd(rddId, blocking) persistentRdds.remove(rddId) listenerBus.post(SparkListenerUnpersistRDD(rddId)) @@ -869,6 +881,7 @@ class SparkContext( dagScheduler = null if (dagSchedulerCopy != null) { metadataCleaner.cancel() + cleaner.foreach(_.stop()) dagSchedulerCopy.stop() listenerBus.stop() taskScheduler = null diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 5ceac28fe7afb..9ea123f174b95 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -180,12 +180,24 @@ object SparkEnv extends Logging { } } + val mapOutputTracker = if (isDriver) { + new MapOutputTrackerMaster(conf) + } else { + new MapOutputTrackerWorker(conf) + } + + // Have to assign trackerActor after initialization as MapOutputTrackerActor + // requires the MapOutputTracker itself + mapOutputTracker.trackerActor = registerOrLookup( + "MapOutputTracker", + new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf)) + val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, - serializer, conf, securityManager) + serializer, conf, securityManager, mapOutputTracker) val connectionManager = blockManager.connectionManager @@ -193,17 +205,6 @@ object SparkEnv extends Logging { val cacheManager = new CacheManager(blockManager) - // Have to assign trackerActor after initialization as MapOutputTrackerActor - // requires the MapOutputTracker itself - val mapOutputTracker = if (isDriver) { - new MapOutputTrackerMaster(conf) - } else { - new MapOutputTracker(conf) - } - mapOutputTracker.trackerActor = registerOrLookup( - "MapOutputTracker", - 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/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index e3c3a12d16f2a..738a3b1bed7f3 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -18,9 +18,8 @@ package org.apache.spark.broadcast import java.io.Serializable -import java.util.concurrent.atomic.AtomicLong -import org.apache.spark._ +import org.apache.spark.SparkException /** * A broadcast variable. Broadcast variables allow the programmer to keep a read-only variable @@ -29,7 +28,8 @@ import org.apache.spark._ * attempts to distribute broadcast variables using efficient broadcast algorithms to reduce * communication cost. * - * Broadcast variables are created from a variable `v` by calling [[SparkContext#broadcast]]. + * Broadcast variables are created from a variable `v` by calling + * [[org.apache.spark.SparkContext#broadcast]]. * The broadcast variable is a wrapper around `v`, and its value can be accessed by calling the * `value` method. The interpreter session below shows this: * @@ -51,49 +51,80 @@ import org.apache.spark._ * @tparam T Type of the data contained in the broadcast variable. */ abstract class Broadcast[T](val id: Long) extends Serializable { - def value: T - // We cannot have an abstract readObject here due to some weird issues with - // readObject having to be 'private' in sub-classes. + /** + * Flag signifying whether the broadcast variable is valid + * (that is, not already destroyed) or not. + */ + @volatile private var _isValid = true - override def toString = "Broadcast(" + id + ")" -} - -private[spark] -class BroadcastManager(val _isDriver: Boolean, conf: SparkConf, securityManager: SecurityManager) - extends Logging with Serializable { - - private var initialized = false - private var broadcastFactory: BroadcastFactory = null - - initialize() - - // Called by SparkContext or Executor before using Broadcast - private def initialize() { - synchronized { - if (!initialized) { - val broadcastFactoryClass = conf.get( - "spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") - - broadcastFactory = - Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] + /** Get the broadcasted value. */ + def value: T = { + assertValid() + getValue() + } - // Initialize appropriate BroadcastFactory and BroadcastObject - broadcastFactory.initialize(isDriver, conf, securityManager) + /** + * Asynchronously delete cached copies of this broadcast on the executors. + * If the broadcast is used after this is called, it will need to be re-sent to each executor. + */ + def unpersist() { + unpersist(blocking = false) + } - initialized = true - } - } + /** + * Delete cached copies of this broadcast on the executors. If the broadcast is used after + * this is called, it will need to be re-sent to each executor. + * @param blocking Whether to block until unpersisting has completed + */ + def unpersist(blocking: Boolean) { + assertValid() + doUnpersist(blocking) } - def stop() { - broadcastFactory.stop() + /** + * Destroy all data and metadata related to this broadcast variable. Use this with caution; + * once a broadcast variable has been destroyed, it cannot be used again. + */ + private[spark] def destroy(blocking: Boolean) { + assertValid() + _isValid = false + doDestroy(blocking) } - private val nextBroadcastId = new AtomicLong(0) + /** + * Whether this Broadcast is actually usable. This should be false once persisted state is + * removed from the driver. + */ + private[spark] def isValid: Boolean = { + _isValid + } - def newBroadcast[T](value_ : T, isLocal: Boolean) = - broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement()) + /** + * Actually get the broadcasted value. Concrete implementations of Broadcast class must + * define their own way to get the value. + */ + private[spark] def getValue(): T + + /** + * Actually unpersist the broadcasted value on the executors. Concrete implementations of + * Broadcast class must define their own logic to unpersist their own data. + */ + private[spark] def doUnpersist(blocking: Boolean) + + /** + * Actually destroy all data and metadata related to this broadcast variable. + * Implementation of Broadcast class must define their own logic to destroy their own + * state. + */ + private[spark] def doDestroy(blocking: Boolean) + + /** Check if this broadcast is valid. If not valid, exception is thrown. */ + private[spark] def assertValid() { + if (!_isValid) { + throw new SparkException("Attempted to use %s after it has been destroyed!".format(toString)) + } + } - def isDriver = _isDriver + override def toString = "Broadcast(" + id + ")" } diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index 6beecaeced5be..c7f7c59cfb449 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -27,7 +27,8 @@ import org.apache.spark.SparkConf * entire Spark job. */ trait BroadcastFactory { - def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit + def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] + def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean): Unit def stop(): Unit } diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala new file mode 100644 index 0000000000000..cf62aca4d45e8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala @@ -0,0 +1,66 @@ +/* + * 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.broadcast + +import java.util.concurrent.atomic.AtomicLong + +import org.apache.spark._ + +private[spark] class BroadcastManager( + val isDriver: Boolean, + conf: SparkConf, + securityManager: SecurityManager) + extends Logging { + + private var initialized = false + private var broadcastFactory: BroadcastFactory = null + + initialize() + + // Called by SparkContext or Executor before using Broadcast + private def initialize() { + synchronized { + if (!initialized) { + val broadcastFactoryClass = + conf.get("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") + + broadcastFactory = + Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] + + // Initialize appropriate BroadcastFactory and BroadcastObject + broadcastFactory.initialize(isDriver, conf, securityManager) + + initialized = true + } + } + } + + def stop() { + broadcastFactory.stop() + } + + private val nextBroadcastId = new AtomicLong(0) + + def newBroadcast[T](value_ : T, isLocal: Boolean) = { + broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement()) + } + + def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) { + broadcastFactory.unbroadcast(id, removeFromDriver, blocking) + } +} diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index e8eb04bb10469..f6a8a8af91e4b 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -17,34 +17,65 @@ package org.apache.spark.broadcast -import java.io.{File, FileOutputStream, ObjectInputStream, OutputStream} -import java.net.{URL, URLConnection, URI} +import java.io.{File, FileOutputStream, ObjectInputStream, ObjectOutputStream, OutputStream} +import java.net.{URI, URL, URLConnection} import java.util.concurrent.TimeUnit -import it.unimi.dsi.fastutil.io.FastBufferedInputStream -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream +import it.unimi.dsi.fastutil.io.{FastBufferedInputStream, FastBufferedOutputStream} -import org.apache.spark.{SparkConf, HttpServer, Logging, SecurityManager, SparkEnv} +import org.apache.spark.{HttpServer, Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashSet, Utils} +/** + * A [[org.apache.spark.broadcast.Broadcast]] implementation that uses HTTP server + * as a broadcast mechanism. The first time a HTTP broadcast variable (sent as part of a + * task) is deserialized in the executor, the broadcasted data is fetched from the driver + * (through a HTTP server running at the driver) and stored in the BlockManager of the + * executor to speed up future accesses. + */ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) with Logging with Serializable { - def value = value_ + def getValue = value_ - def blockId = BroadcastBlockId(id) + val blockId = BroadcastBlockId(id) + /* + * Broadcasted data is also stored in the BlockManager of the driver. The BlockManagerMaster + * does not need to be told about this block as not only need to know about this data block. + */ HttpBroadcast.synchronized { - SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false) + SparkEnv.get.blockManager.putSingle( + blockId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) } if (!isLocal) { HttpBroadcast.write(id, value_) } - // Called by JVM when deserializing an object + /** + * Remove all persisted state associated with this HTTP broadcast on the executors. + */ + def doUnpersist(blocking: Boolean) { + HttpBroadcast.unpersist(id, removeFromDriver = false, blocking) + } + + /** + * Remove all persisted state associated with this HTTP broadcast on the executors and driver. + */ + def doDestroy(blocking: Boolean) { + HttpBroadcast.unpersist(id, removeFromDriver = true, blocking) + } + + /** Used by the JVM when serializing this object. */ + private def writeObject(out: ObjectOutputStream) { + assertValid() + out.defaultWriteObject() + } + + /** Used by the JVM when deserializing this object. */ private def readObject(in: ObjectInputStream) { in.defaultReadObject() HttpBroadcast.synchronized { @@ -54,7 +85,13 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea logInfo("Started reading broadcast variable " + id) val start = System.nanoTime value_ = HttpBroadcast.read[T](id) - SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false) + /* + * We cache broadcast data in the BlockManager so that subsequent tasks using it + * do not need to re-fetch. This data is only used locally and no other node + * needs to fetch this block, so we don't notify the master. + */ + SparkEnv.get.blockManager.putSingle( + blockId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) val time = (System.nanoTime - start) / 1e9 logInfo("Reading broadcast variable " + id + " took " + time + " s") } @@ -63,23 +100,8 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea } } -/** - * A [[BroadcastFactory]] implementation that uses a HTTP server as the broadcast medium. - */ -class HttpBroadcastFactory extends BroadcastFactory { - def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { - HttpBroadcast.initialize(isDriver, conf, securityMgr) - } - - def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = - new HttpBroadcast[T](value_, isLocal, id) - - def stop() { HttpBroadcast.stop() } -} - -private object HttpBroadcast extends Logging { +private[spark] object HttpBroadcast extends Logging { private var initialized = false - private var broadcastDir: File = null private var compress: Boolean = false private var bufferSize: Int = 65536 @@ -89,11 +111,9 @@ private object HttpBroadcast extends Logging { // TODO: This shouldn't be a global variable so that multiple SparkContexts can coexist private val files = new TimeStampedHashSet[String] - private var cleaner: MetadataCleaner = null - private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES).toInt - private var compressionCodec: CompressionCodec = null + private var cleaner: MetadataCleaner = null def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { synchronized { @@ -136,8 +156,10 @@ private object HttpBroadcast extends Logging { logInfo("Broadcast server started at " + serverUri) } + def getFile(id: Long) = new File(broadcastDir, BroadcastBlockId(id).name) + def write(id: Long, value: Any) { - val file = new File(broadcastDir, BroadcastBlockId(id).name) + val file = getFile(id) val out: OutputStream = { if (compress) { compressionCodec.compressedOutputStream(new FileOutputStream(file)) @@ -160,7 +182,7 @@ private object HttpBroadcast extends Logging { if (securityManager.isAuthenticationEnabled()) { logDebug("broadcast security enabled") val newuri = Utils.constructURIForAuthentication(new URI(url), securityManager) - uc = newuri.toURL().openConnection() + uc = newuri.toURL.openConnection() uc.setAllowUserInteraction(false) } else { logDebug("broadcast not using security") @@ -169,7 +191,7 @@ private object HttpBroadcast extends Logging { val in = { uc.setReadTimeout(httpReadTimeout) - val inputStream = uc.getInputStream(); + val inputStream = uc.getInputStream if (compress) { compressionCodec.compressedInputStream(inputStream) } else { @@ -183,20 +205,48 @@ private object HttpBroadcast extends Logging { obj } - def cleanup(cleanupTime: Long) { + /** + * Remove all persisted blocks associated with this HTTP broadcast on the executors. + * If removeFromDriver is true, also remove these persisted blocks on the driver + * and delete the associated broadcast file. + */ + def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = synchronized { + SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) + if (removeFromDriver) { + val file = getFile(id) + files.remove(file.toString) + deleteBroadcastFile(file) + } + } + + /** + * Periodically clean up old broadcasts by removing the associated map entries and + * deleting the associated files. + */ + private def cleanup(cleanupTime: Long) { val iterator = files.internalMap.entrySet().iterator() while(iterator.hasNext) { val entry = iterator.next() val (file, time) = (entry.getKey, entry.getValue) if (time < cleanupTime) { - try { - iterator.remove() - new File(file.toString).delete() - logInfo("Deleted broadcast file '" + file + "'") - } catch { - case e: Exception => logWarning("Could not delete broadcast file '" + file + "'", e) + iterator.remove() + deleteBroadcastFile(new File(file.toString)) + } + } + } + + private def deleteBroadcastFile(file: File) { + try { + if (file.exists) { + if (file.delete()) { + logInfo("Deleted broadcast file: %s".format(file)) + } else { + logWarning("Could not delete broadcast file: %s".format(file)) } } + } catch { + case e: Exception => + logError("Exception while deleting broadcast file: %s".format(file), e) } } } diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala new file mode 100644 index 0000000000000..e3f6cdc6154dd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala @@ -0,0 +1,45 @@ +/* + * 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.broadcast + +import org.apache.spark.{SecurityManager, SparkConf} + +/** + * A [[org.apache.spark.broadcast.BroadcastFactory]] implementation that uses a + * HTTP server as the broadcast mechanism. Refer to + * [[org.apache.spark.broadcast.HttpBroadcast]] for more details about this mechanism. + */ +class HttpBroadcastFactory extends BroadcastFactory { + def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { + HttpBroadcast.initialize(isDriver, conf, securityMgr) + } + + def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = + new HttpBroadcast[T](value_, isLocal, id) + + def stop() { HttpBroadcast.stop() } + + /** + * Remove all persisted state associated with the HTTP broadcast with the given ID. + * @param removeFromDriver Whether to remove state from the driver + * @param blocking Whether to block until unbroadcasted + */ + def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) { + HttpBroadcast.unpersist(id, removeFromDriver, blocking) + } +} diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 2595c15104e87..2b32546c6854d 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -17,24 +17,43 @@ package org.apache.spark.broadcast -import java.io._ +import java.io.{ByteArrayInputStream, ObjectInputStream, ObjectOutputStream} import scala.math import scala.util.Random -import org.apache.spark._ -import org.apache.spark.storage.{BroadcastBlockId, BroadcastHelperBlockId, StorageLevel} +import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException} +import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} import org.apache.spark.util.Utils +/** + * A [[org.apache.spark.broadcast.Broadcast]] implementation that uses a BitTorrent-like + * protocol to do a distributed transfer of the broadcasted data to the executors. + * The mechanism is as follows. The driver divides the serializes the broadcasted data, + * divides it into smaller chunks, and stores them in the BlockManager of the driver. + * These chunks are reported to the BlockManagerMaster so that all the executors can + * learn the location of those chunks. The first time the broadcast variable (sent as + * part of task) is deserialized at a executor, all the chunks are fetched using + * the BlockManager. When all the chunks are fetched (initially from the driver's + * BlockManager), they are combined and deserialized to recreate the broadcasted data. + * However, the chunks are also stored in the BlockManager and reported to the + * BlockManagerMaster. As more executors fetch the chunks, BlockManagerMaster learns + * multiple locations for each chunk. Hence, subsequent fetches of each chunk will be + * made to other executors who already have those chunks, resulting in a distributed + * fetching. This prevents the driver from being the bottleneck in sending out multiple + * copies of the broadcast data (one per executor) as done by the + * [[org.apache.spark.broadcast.HttpBroadcast]]. + */ private[spark] class TorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) -extends Broadcast[T](id) with Logging with Serializable { + extends Broadcast[T](id) with Logging with Serializable { - def value = value_ + def getValue = value_ - def broadcastId = BroadcastBlockId(id) + val broadcastId = BroadcastBlockId(id) TorrentBroadcast.synchronized { - SparkEnv.get.blockManager.putSingle(broadcastId, value_, StorageLevel.MEMORY_AND_DISK, false) + SparkEnv.get.blockManager.putSingle( + broadcastId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) } @transient var arrayOfBlocks: Array[TorrentBlock] = null @@ -46,32 +65,52 @@ extends Broadcast[T](id) with Logging with Serializable { sendBroadcast() } - def sendBroadcast() { - var tInfo = TorrentBroadcast.blockifyObject(value_) + /** + * Remove all persisted state associated with this Torrent broadcast on the executors. + */ + def doUnpersist(blocking: Boolean) { + TorrentBroadcast.unpersist(id, removeFromDriver = false, blocking) + } + + /** + * Remove all persisted state associated with this Torrent broadcast on the executors + * and driver. + */ + def doDestroy(blocking: Boolean) { + TorrentBroadcast.unpersist(id, removeFromDriver = true, blocking) + } + def sendBroadcast() { + val tInfo = TorrentBroadcast.blockifyObject(value_) totalBlocks = tInfo.totalBlocks totalBytes = tInfo.totalBytes hasBlocks = tInfo.totalBlocks // Store meta-info - val metaId = BroadcastHelperBlockId(broadcastId, "meta") + val metaId = BroadcastBlockId(id, "meta") val metaInfo = TorrentInfo(null, totalBlocks, totalBytes) TorrentBroadcast.synchronized { SparkEnv.get.blockManager.putSingle( - metaId, metaInfo, StorageLevel.MEMORY_AND_DISK, true) + metaId, metaInfo, StorageLevel.MEMORY_AND_DISK, tellMaster = true) } // Store individual pieces for (i <- 0 until totalBlocks) { - val pieceId = BroadcastHelperBlockId(broadcastId, "piece" + i) + val pieceId = BroadcastBlockId(id, "piece" + i) TorrentBroadcast.synchronized { SparkEnv.get.blockManager.putSingle( - pieceId, tInfo.arrayOfBlocks(i), StorageLevel.MEMORY_AND_DISK, true) + pieceId, tInfo.arrayOfBlocks(i), StorageLevel.MEMORY_AND_DISK, tellMaster = true) } } } - // Called by JVM when deserializing an object + /** Used by the JVM when serializing this object. */ + private def writeObject(out: ObjectOutputStream) { + assertValid() + out.defaultWriteObject() + } + + /** Used by the JVM when deserializing this object. */ private def readObject(in: ObjectInputStream) { in.defaultReadObject() TorrentBroadcast.synchronized { @@ -86,18 +125,22 @@ extends Broadcast[T](id) with Logging with Serializable { // Initialize @transient variables that will receive garbage values from the master. resetWorkerVariables() - if (receiveBroadcast(id)) { + if (receiveBroadcast()) { value_ = TorrentBroadcast.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks) - // Store the merged copy in cache so that the next worker doesn't need to rebuild it. - // This creates a tradeoff between memory usage and latency. - // Storing copy doubles the memory footprint; not storing doubles deserialization cost. + /* Store the merged copy in cache so that the next worker doesn't need to rebuild it. + * This creates a trade-off between memory usage and latency. Storing copy doubles + * the memory footprint; not storing doubles deserialization cost. Also, + * this does not need to be reported to BlockManagerMaster since other executors + * does not need to access this block (they only need to fetch the chunks, + * which are reported). + */ SparkEnv.get.blockManager.putSingle( - broadcastId, value_, StorageLevel.MEMORY_AND_DISK, false) + broadcastId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) // Remove arrayOfBlocks from memory once value_ is on local cache resetWorkerVariables() - } else { + } else { logError("Reading broadcast variable " + id + " failed") } @@ -114,9 +157,10 @@ extends Broadcast[T](id) with Logging with Serializable { hasBlocks = 0 } - def receiveBroadcast(variableID: Long): Boolean = { - // Receive meta-info - val metaId = BroadcastHelperBlockId(broadcastId, "meta") + def receiveBroadcast(): Boolean = { + // Receive meta-info about the size of broadcast data, + // the number of chunks it is divided into, etc. + val metaId = BroadcastBlockId(id, "meta") var attemptId = 10 while (attemptId > 0 && totalBlocks == -1) { TorrentBroadcast.synchronized { @@ -138,17 +182,21 @@ extends Broadcast[T](id) with Logging with Serializable { return false } - // Receive actual blocks + /* + * Fetch actual chunks of data. Note that all these chunks are stored in + * the BlockManager and reported to the master, so that other executors + * can find out and pull the chunks from this executor. + */ val recvOrder = new Random().shuffle(Array.iterate(0, totalBlocks)(_ + 1).toList) for (pid <- recvOrder) { - val pieceId = BroadcastHelperBlockId(broadcastId, "piece" + pid) + val pieceId = BroadcastBlockId(id, "piece" + pid) TorrentBroadcast.synchronized { SparkEnv.get.blockManager.getSingle(pieceId) match { case Some(x) => arrayOfBlocks(pid) = x.asInstanceOf[TorrentBlock] hasBlocks += 1 SparkEnv.get.blockManager.putSingle( - pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, true) + pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, tellMaster = true) case None => throw new SparkException("Failed to get " + pieceId + " of " + broadcastId) @@ -156,16 +204,16 @@ extends Broadcast[T](id) with Logging with Serializable { } } - (hasBlocks == totalBlocks) + hasBlocks == totalBlocks } } -private object TorrentBroadcast -extends Logging { - +private[spark] object TorrentBroadcast extends Logging { + private lazy val BLOCK_SIZE = conf.getInt("spark.broadcast.blockSize", 4096) * 1024 private var initialized = false private var conf: SparkConf = null + def initialize(_isDriver: Boolean, conf: SparkConf) { TorrentBroadcast.conf = conf // TODO: we might have to fix it in tests synchronized { @@ -179,39 +227,37 @@ extends Logging { initialized = false } - lazy val BLOCK_SIZE = conf.getInt("spark.broadcast.blockSize", 4096) * 1024 - def blockifyObject[T](obj: T): TorrentInfo = { val byteArray = Utils.serialize[T](obj) val bais = new ByteArrayInputStream(byteArray) - var blockNum = (byteArray.length / BLOCK_SIZE) + var blockNum = byteArray.length / BLOCK_SIZE if (byteArray.length % BLOCK_SIZE != 0) { blockNum += 1 } - var retVal = new Array[TorrentBlock](blockNum) - var blockID = 0 + val blocks = new Array[TorrentBlock](blockNum) + var blockId = 0 for (i <- 0 until (byteArray.length, BLOCK_SIZE)) { val thisBlockSize = math.min(BLOCK_SIZE, byteArray.length - i) - var tempByteArray = new Array[Byte](thisBlockSize) - val hasRead = bais.read(tempByteArray, 0, thisBlockSize) + val tempByteArray = new Array[Byte](thisBlockSize) + bais.read(tempByteArray, 0, thisBlockSize) - retVal(blockID) = new TorrentBlock(blockID, tempByteArray) - blockID += 1 + blocks(blockId) = new TorrentBlock(blockId, tempByteArray) + blockId += 1 } bais.close() - val tInfo = TorrentInfo(retVal, blockNum, byteArray.length) - tInfo.hasBlocks = blockNum - - tInfo + val info = TorrentInfo(blocks, blockNum, byteArray.length) + info.hasBlocks = blockNum + info } - def unBlockifyObject[T](arrayOfBlocks: Array[TorrentBlock], - totalBytes: Int, - totalBlocks: Int): T = { + def unBlockifyObject[T]( + arrayOfBlocks: Array[TorrentBlock], + totalBytes: Int, + totalBlocks: Int): T = { val retByteArray = new Array[Byte](totalBytes) for (i <- 0 until totalBlocks) { System.arraycopy(arrayOfBlocks(i).byteArray, 0, retByteArray, @@ -220,6 +266,13 @@ extends Logging { Utils.deserialize[T](retByteArray, Thread.currentThread.getContextClassLoader) } + /** + * Remove all persisted blocks associated with this torrent broadcast on the executors. + * If removeFromDriver is true, also remove these persisted blocks on the driver. + */ + def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = synchronized { + SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) + } } private[spark] case class TorrentBlock( @@ -228,25 +281,10 @@ private[spark] case class TorrentBlock( extends Serializable private[spark] case class TorrentInfo( - @transient arrayOfBlocks : Array[TorrentBlock], + @transient arrayOfBlocks: Array[TorrentBlock], totalBlocks: Int, totalBytes: Int) extends Serializable { @transient var hasBlocks = 0 } - -/** - * A [[BroadcastFactory]] that creates a torrent-based implementation of broadcast. - */ -class TorrentBroadcastFactory extends BroadcastFactory { - - def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { - TorrentBroadcast.initialize(isDriver, conf) - } - - def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = - new TorrentBroadcast[T](value_, isLocal, id) - - def stop() { TorrentBroadcast.stop() } -} diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala new file mode 100644 index 0000000000000..d216b58718148 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala @@ -0,0 +1,46 @@ +/* + * 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.broadcast + +import org.apache.spark.{SecurityManager, SparkConf} + +/** + * A [[org.apache.spark.broadcast.Broadcast]] implementation that uses a BitTorrent-like + * protocol to do a distributed transfer of the broadcasted data to the executors. Refer to + * [[org.apache.spark.broadcast.TorrentBroadcast]] for more details. + */ +class TorrentBroadcastFactory extends BroadcastFactory { + + def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { + TorrentBroadcast.initialize(isDriver, conf) + } + + def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = + new TorrentBroadcast[T](value_, isLocal, id) + + def stop() { TorrentBroadcast.stop() } + + /** + * Remove all persisted state associated with the torrent broadcast with the given ID. + * @param removeFromDriver Whether to remove state from the driver. + * @param blocking Whether to block until unbroadcasted + */ + def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) { + TorrentBroadcast.unpersist(id, removeFromDriver, blocking) + } +} diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 6b0a972f0bbe0..bdf586351ac14 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -17,7 +17,6 @@ package org.apache.spark.network -import java.net._ import java.nio._ import java.nio.channels._ import java.nio.channels.spi._ 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 c43823bd769b7..bf3c57ad41eb2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -138,6 +138,8 @@ abstract class RDD[T: ClassTag]( "Cannot change storage level of an RDD after it was already assigned a level") } sc.persistRDD(this) + // Register the RDD with the ContextCleaner for automatic GC-based cleanup + sc.cleaner.foreach(_.registerRDDForCleanup(this)) storageLevel = newLevel this } @@ -156,7 +158,7 @@ abstract class RDD[T: ClassTag]( */ def unpersist(blocking: Boolean = true): RDD[T] = { logInfo("Removing RDD " + id + " from persistence list") - sc.unpersistRDD(this, blocking) + sc.unpersistRDD(id, blocking) storageLevel = StorageLevel.NONE this } @@ -1141,5 +1143,4 @@ abstract class RDD[T: ClassTag]( def toJavaRDD() : JavaRDD[T] = { new JavaRDD(this)(elementClassTag) } - } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 442a95bb2c44b..6368665f249ee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -32,7 +32,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerMaster, RDDBlockId} -import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} +import org.apache.spark.util.Utils /** * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of @@ -80,13 +80,13 @@ class DAGScheduler( private[scheduler] def numTotalJobs: Int = nextJobId.get() private val nextStageId = new AtomicInteger(0) - private[scheduler] val jobIdToStageIds = new TimeStampedHashMap[Int, HashSet[Int]] - private[scheduler] val stageIdToJobIds = new TimeStampedHashMap[Int, HashSet[Int]] - private[scheduler] val stageIdToStage = new TimeStampedHashMap[Int, Stage] - private[scheduler] val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] + private[scheduler] val jobIdToStageIds = new HashMap[Int, HashSet[Int]] + private[scheduler] val stageIdToJobIds = new HashMap[Int, HashSet[Int]] + private[scheduler] val stageIdToStage = new HashMap[Int, Stage] + private[scheduler] val shuffleToMapStage = new HashMap[Int, Stage] private[scheduler] val jobIdToActiveJob = new HashMap[Int, ActiveJob] private[scheduler] val resultStageToJob = new HashMap[Stage, ActiveJob] - private[scheduler] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] + private[scheduler] val stageToInfos = new HashMap[Stage, StageInfo] // Stages we need to run whose parents aren't done private[scheduler] val waitingStages = new HashSet[Stage] @@ -98,7 +98,7 @@ class DAGScheduler( private[scheduler] val failedStages = new HashSet[Stage] // Missing tasks from each stage - private[scheduler] val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] + private[scheduler] val pendingTasks = new HashMap[Stage, HashSet[Task[_]]] private[scheduler] val activeJobs = new HashSet[ActiveJob] @@ -113,9 +113,6 @@ class DAGScheduler( // stray messages to detect. private val failedEpoch = new HashMap[String, Long] - private val metadataCleaner = - new MetadataCleaner(MetadataCleanerType.DAG_SCHEDULER, this.cleanup, env.conf) - taskScheduler.setDAGScheduler(this) /** @@ -258,7 +255,7 @@ class DAGScheduler( : Stage = { val stage = newStage(rdd, numTasks, Some(shuffleDep), jobId, callSite) - if (mapOutputTracker.has(shuffleDep.shuffleId)) { + if (mapOutputTracker.containsShuffle(shuffleDep.shuffleId)) { val serLocs = mapOutputTracker.getSerializedMapOutputStatuses(shuffleDep.shuffleId) val locs = MapOutputTracker.deserializeMapStatuses(serLocs) for (i <- 0 until locs.size) { @@ -390,6 +387,9 @@ class DAGScheduler( stageIdToStage -= stageId stageIdToJobIds -= stageId + ShuffleMapTask.removeStage(stageId) + ResultTask.removeStage(stageId) + logDebug("After removal of stage %d, remaining stages = %d" .format(stageId, stageIdToStage.size)) } @@ -1084,26 +1084,10 @@ class DAGScheduler( Nil } - private def cleanup(cleanupTime: Long) { - Map( - "stageIdToStage" -> stageIdToStage, - "shuffleToMapStage" -> shuffleToMapStage, - "pendingTasks" -> pendingTasks, - "stageToInfos" -> stageToInfos, - "jobIdToStageIds" -> jobIdToStageIds, - "stageIdToJobIds" -> stageIdToJobIds). - foreach { case (s, t) => - val sizeBefore = t.size - t.clearOldValues(cleanupTime) - logInfo("%s %d --> %d".format(s, sizeBefore, t.size)) - } - } - def stop() { if (eventProcessActor != null) { eventProcessActor ! StopDAGScheduler } - metadataCleaner.cancel() taskScheduler.stop() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 3fc6cc9850feb..083fb895d8696 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -20,21 +20,17 @@ package org.apache.spark.scheduler import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} +import scala.collection.mutable.HashMap + import org.apache.spark._ -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.RDDCheckpointData -import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} +import org.apache.spark.rdd.{RDD, RDDCheckpointData} private[spark] object ResultTask { // A simple map between the stage id to the serialized byte array of a task. // Served as a cache for task serialization because serialization can be // expensive on the master node if it needs to launch thousands of tasks. - val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]] - - // TODO: This object shouldn't have global variables - val metadataCleaner = new MetadataCleaner( - MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues, new SparkConf) + private val serializedInfoCache = new HashMap[Int, Array[Byte]] def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = { @@ -67,6 +63,10 @@ private[spark] object ResultTask { (rdd, func) } + def removeStage(stageId: Int) { + serializedInfoCache.remove(stageId) + } + def clearCache() { synchronized { serializedInfoCache.clear() 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 2a9edf4a76b97..23f3b3e824762 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -24,22 +24,16 @@ import scala.collection.mutable.HashMap 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.rdd.{RDD, RDDCheckpointData} import org.apache.spark.serializer.Serializer import org.apache.spark.storage._ -import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} private[spark] object ShuffleMapTask { // A simple map between the stage id to the serialized byte array of a task. // Served as a cache for task serialization because serialization can be // expensive on the master node if it needs to launch thousands of tasks. - val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]] - - // TODO: This object shouldn't have global variables - val metadataCleaner = new MetadataCleaner( - MetadataCleanerType.SHUFFLE_MAP_TASK, serializedInfoCache.clearOldValues, new SparkConf) + private val serializedInfoCache = new HashMap[Int, Array[Byte]] def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_]): Array[Byte] = { synchronized { @@ -80,6 +74,10 @@ private[spark] object ShuffleMapTask { HashMap(set.toSeq: _*) } + def removeStage(stageId: Int) { + serializedInfoCache.remove(stageId) + } + def clearCache() { synchronized { serializedInfoCache.clear() diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index a92922166f595..acd152dda89d4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -42,7 +42,7 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode * * THREADING: SchedulerBackends and task-submitting clients can call this class from multiple * threads, so it needs locks in public API methods to maintain its state. In addition, some - * SchedulerBackends sycnchronize on themselves when they want to send events here, and then + * SchedulerBackends synchronize on themselves when they want to send events here, and then * acquire a lock on us, so we need to make sure that we don't try to lock the backend while * we are holding a lock on ourselves. */ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 301d784b350a3..cffea28fbf794 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -34,7 +34,7 @@ private[spark] sealed abstract class BlockId { def asRDDId = if (isRDD) Some(asInstanceOf[RDDBlockId]) else None def isRDD = isInstanceOf[RDDBlockId] def isShuffle = isInstanceOf[ShuffleBlockId] - def isBroadcast = isInstanceOf[BroadcastBlockId] || isInstanceOf[BroadcastHelperBlockId] + def isBroadcast = isInstanceOf[BroadcastBlockId] override def toString = name override def hashCode = name.hashCode @@ -48,18 +48,13 @@ private[spark] case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockI def name = "rdd_" + rddId + "_" + splitIndex } -private[spark] -case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { +private[spark] case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) + extends BlockId { def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId } -private[spark] case class BroadcastBlockId(broadcastId: Long) extends BlockId { - def name = "broadcast_" + broadcastId -} - -private[spark] -case class BroadcastHelperBlockId(broadcastId: BroadcastBlockId, hType: String) extends BlockId { - def name = broadcastId.name + "_" + hType +private[spark] case class BroadcastBlockId(broadcastId: Long, field: String = "") extends BlockId { + def name = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field) } private[spark] case class TaskResultBlockId(taskId: Long) extends BlockId { @@ -83,8 +78,7 @@ private[spark] case class TestBlockId(id: String) extends BlockId { private[spark] object BlockId { val RDD = "rdd_([0-9]+)_([0-9]+)".r val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r - val BROADCAST = "broadcast_([0-9]+)".r - val BROADCAST_HELPER = "broadcast_([0-9]+)_([A-Za-z0-9]+)".r + val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r val TASKRESULT = "taskresult_([0-9]+)".r val STREAM = "input-([0-9]+)-([0-9]+)".r val TEST = "test_(.*)".r @@ -95,10 +89,8 @@ private[spark] object BlockId { RDDBlockId(rddId.toInt, splitIndex.toInt) case SHUFFLE(shuffleId, mapId, reduceId) => ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) - case BROADCAST(broadcastId) => - BroadcastBlockId(broadcastId.toLong) - case BROADCAST_HELPER(broadcastId, hType) => - BroadcastHelperBlockId(BroadcastBlockId(broadcastId.toLong), hType) + case BROADCAST(broadcastId, field) => + BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => TaskResultBlockId(taskId.toLong) case STREAM(streamId, uniqueId) => diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 19138d9dde697..b021564477c47 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -19,20 +19,22 @@ package org.apache.spark.storage import java.io.{File, InputStream, OutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} + import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.{Await, Future} import scala.concurrent.duration._ import scala.util.Random + import akka.actor.{ActorSystem, Cancellable, Props} import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} import sun.nio.ch.DirectBuffer -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv, SparkException} + +import org.apache.spark.{Logging, MapOutputTracker, SecurityManager, SparkConf, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer import org.apache.spark.util._ - sealed trait Values case class ByteBufferValues(buffer: ByteBuffer) extends Values @@ -46,7 +48,8 @@ private[spark] class BlockManager( val defaultSerializer: Serializer, maxMemory: Long, val conf: SparkConf, - securityManager: SecurityManager) + securityManager: SecurityManager, + mapOutputTracker: MapOutputTracker) extends Logging { val shuffleBlockManager = new ShuffleBlockManager(this) @@ -55,7 +58,7 @@ private[spark] class BlockManager( private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] - private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory) + private[storage] val memoryStore = new MemoryStore(this, maxMemory) private[storage] val diskStore = new DiskStore(this, diskBlockManager) var tachyonInitialized = false private[storage] lazy val tachyonStore: TachyonStore = { @@ -98,7 +101,7 @@ private[spark] class BlockManager( val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf) - val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)), + val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this, mapOutputTracker)), name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next) // Pending re-registration action being executed asynchronously or null if none @@ -137,9 +140,10 @@ private[spark] class BlockManager( master: BlockManagerMaster, serializer: Serializer, conf: SparkConf, - securityManager: SecurityManager) = { + securityManager: SecurityManager, + mapOutputTracker: MapOutputTracker) = { this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), - conf, securityManager) + conf, securityManager, mapOutputTracker) } /** @@ -217,9 +221,26 @@ private[spark] class BlockManager( } /** - * Get storage level of local block. If no info exists for the block, then returns null. + * Get the BlockStatus for the block identified by the given ID, if it exists. + * NOTE: This is mainly for testing, and it doesn't fetch information from Tachyon. + */ + def getStatus(blockId: BlockId): Option[BlockStatus] = { + blockInfo.get(blockId).map { info => + val memSize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L + val diskSize = if (diskStore.contains(blockId)) diskStore.getSize(blockId) else 0L + // Assume that block is not in Tachyon + BlockStatus(info.level, memSize, diskSize, 0L) + } + } + + /** + * Get the ids of existing blocks that match the given filter. Note that this will + * query the blocks stored in the disk block manager (that the block manager + * may not know of). */ - def getLevel(blockId: BlockId): StorageLevel = blockInfo.get(blockId).map(_.level).orNull + def getMatchingBlockIds(filter: BlockId => Boolean): Seq[BlockId] = { + (blockInfo.keys ++ diskBlockManager.getAllBlocks()).filter(filter).toSeq + } /** * Tell the master about the current storage status of a block. This will send a block update @@ -525,9 +546,8 @@ private[spark] class BlockManager( /** * A short circuited method to get a block writer that can write data directly to disk. - * The Block will be appended to the File specified by filename. - * This is currently used for writing shuffle files out. Callers should handle error - * cases. + * The Block will be appended to the File specified by filename. This is currently used for + * writing shuffle files out. Callers should handle error cases. */ def getDiskWriter( blockId: BlockId, @@ -863,11 +883,22 @@ private[spark] class BlockManager( * @return The number of blocks removed. */ def removeRdd(rddId: Int): Int = { - // TODO: Instead of doing a linear scan on the blockInfo map, create another map that maps - // from RDD.id to blocks. + // TODO: Avoid a linear scan by creating another mapping of RDD.id to blocks. logInfo("Removing RDD " + rddId) val blocksToRemove = blockInfo.keys.flatMap(_.asRDDId).filter(_.rddId == rddId) - blocksToRemove.foreach(blockId => removeBlock(blockId, tellMaster = false)) + blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster = false) } + blocksToRemove.size + } + + /** + * Remove all blocks belonging to the given broadcast. + */ + def removeBroadcast(broadcastId: Long, tellMaster: Boolean): Int = { + logInfo("Removing broadcast " + broadcastId) + val blocksToRemove = blockInfo.keys.collect { + case bid @ BroadcastBlockId(`broadcastId`, _) => bid + } + blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster) } blocksToRemove.size } @@ -908,10 +939,10 @@ private[spark] class BlockManager( } private def dropOldBlocks(cleanupTime: Long, shouldDrop: (BlockId => Boolean)) { - val iterator = blockInfo.internalMap.entrySet().iterator() + val iterator = blockInfo.getEntrySet.iterator while (iterator.hasNext) { val entry = iterator.next() - val (id, info, time) = (entry.getKey, entry.getValue._1, entry.getValue._2) + val (id, info, time) = (entry.getKey, entry.getValue.value, entry.getValue.timestamp) if (time < cleanupTime && shouldDrop(id)) { info.synchronized { val level = info.level @@ -935,7 +966,7 @@ private[spark] class BlockManager( def shouldCompress(blockId: BlockId): Boolean = blockId match { case ShuffleBlockId(_, _, _) => compressShuffle - case BroadcastBlockId(_) => compressBroadcast + case BroadcastBlockId(_, _) => compressBroadcast case RDDBlockId(_, _) => compressRdds case TempBlockId(_) => compressShuffleSpill case _ => false diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 4bc1b407ad106..7897fade2df2b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -81,6 +81,14 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log askDriverWithReply[Seq[Seq[BlockManagerId]]](GetLocationsMultipleBlockIds(blockIds)) } + /** + * Check if block manager master has a block. Note that this can be used to check for only + * those blocks that are reported to block manager master. + */ + def contains(blockId: BlockId) = { + !getLocations(blockId).isEmpty + } + /** Get ids of other nodes in the cluster from the driver */ def getPeers(blockManagerId: BlockManagerId, numPeers: Int): Seq[BlockManagerId] = { val result = askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId, numPeers)) @@ -99,12 +107,10 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log askDriverWithReply(RemoveBlock(blockId)) } - /** - * Remove all blocks belonging to the given RDD. - */ + /** Remove all blocks belonging to the given RDD. */ def removeRdd(rddId: Int, blocking: Boolean) { val future = askDriverWithReply[Future[Seq[Int]]](RemoveRdd(rddId)) - future onFailure { + future.onFailure { case e: Throwable => logError("Failed to remove RDD " + rddId, e) } if (blocking) { @@ -112,6 +118,31 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log } } + /** Remove all blocks belonging to the given shuffle. */ + def removeShuffle(shuffleId: Int, blocking: Boolean) { + val future = askDriverWithReply[Future[Seq[Boolean]]](RemoveShuffle(shuffleId)) + future.onFailure { + case e: Throwable => logError("Failed to remove shuffle " + shuffleId, e) + } + if (blocking) { + Await.result(future, timeout) + } + } + + /** Remove all blocks belonging to the given broadcast. */ + def removeBroadcast(broadcastId: Long, removeFromMaster: Boolean, blocking: Boolean) { + val future = askDriverWithReply[Future[Seq[Int]]]( + RemoveBroadcast(broadcastId, removeFromMaster)) + future.onFailure { + case e: Throwable => + logError("Failed to remove broadcast " + broadcastId + + " with removeFromMaster = " + removeFromMaster, e) + } + if (blocking) { + Await.result(future, timeout) + } + } + /** * Return the memory status for each block manager, in the form of a map from * the block manager's id to two long values. The first value is the maximum @@ -126,6 +157,51 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log askDriverWithReply[Array[StorageStatus]](GetStorageStatus) } + /** + * Return the block's status on all block managers, if any. NOTE: This is a + * potentially expensive operation and should only be used for testing. + * + * If askSlaves is true, this invokes the master to query each block manager for the most + * updated block statuses. This is useful when the master is not informed of the given block + * by all block managers. + */ + def getBlockStatus( + blockId: BlockId, + askSlaves: Boolean = true): Map[BlockManagerId, BlockStatus] = { + val msg = GetBlockStatus(blockId, askSlaves) + /* + * To avoid potential deadlocks, the use of Futures is necessary, because the master actor + * should not block on waiting for a block manager, which can in turn be waiting for the + * master actor for a response to a prior message. + */ + val response = askDriverWithReply[Map[BlockManagerId, Future[Option[BlockStatus]]]](msg) + val (blockManagerIds, futures) = response.unzip + val result = Await.result(Future.sequence(futures), timeout) + if (result == null) { + throw new SparkException("BlockManager returned null for BlockStatus query: " + blockId) + } + val blockStatus = result.asInstanceOf[Iterable[Option[BlockStatus]]] + blockManagerIds.zip(blockStatus).flatMap { case (blockManagerId, status) => + status.map { s => (blockManagerId, s) } + }.toMap + } + + /** + * Return a list of ids of existing blocks such that the ids match the given filter. NOTE: This + * is a potentially expensive operation and should only be used for testing. + * + * If askSlaves is true, this invokes the master to query each block manager for the most + * updated block statuses. This is useful when the master is not informed of the given block + * by all block managers. + */ + def getMatchingBlockIds( + filter: BlockId => Boolean, + askSlaves: Boolean): Seq[BlockId] = { + val msg = GetMatchingBlockIds(filter, askSlaves) + val future = askDriverWithReply[Future[Seq[BlockId]]](msg) + Await.result(future, timeout) + } + /** Stop the driver actor, called only on the Spark driver node */ def stop() { if (driverActor != null) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 378f4cadc17d7..c57b6e8391b13 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -94,9 +94,21 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus case GetStorageStatus => sender ! storageStatus + case GetBlockStatus(blockId, askSlaves) => + sender ! blockStatus(blockId, askSlaves) + + case GetMatchingBlockIds(filter, askSlaves) => + sender ! getMatchingBlockIds(filter, askSlaves) + case RemoveRdd(rddId) => sender ! removeRdd(rddId) + case RemoveShuffle(shuffleId) => + sender ! removeShuffle(shuffleId) + + case RemoveBroadcast(broadcastId, removeFromDriver) => + sender ! removeBroadcast(broadcastId, removeFromDriver) + case RemoveBlock(blockId) => removeBlockFromWorkers(blockId) sender ! true @@ -140,9 +152,41 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus // The dispatcher is used as an implicit argument into the Future sequence construction. import context.dispatcher val removeMsg = RemoveRdd(rddId) - Future.sequence(blockManagerInfo.values.map { bm => - bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] - }.toSeq) + Future.sequence( + blockManagerInfo.values.map { bm => + bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] + }.toSeq + ) + } + + private def removeShuffle(shuffleId: Int): Future[Seq[Boolean]] = { + // Nothing to do in the BlockManagerMasterActor data structures + import context.dispatcher + val removeMsg = RemoveShuffle(shuffleId) + Future.sequence( + blockManagerInfo.values.map { bm => + bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Boolean] + }.toSeq + ) + } + + /** + * Delegate RemoveBroadcast messages to each BlockManager because the master may not notified + * of all broadcast blocks. If removeFromDriver is false, broadcast blocks are only removed + * from the executors, but not from the driver. + */ + private def removeBroadcast(broadcastId: Long, removeFromDriver: Boolean): Future[Seq[Int]] = { + // TODO: Consolidate usages of + import context.dispatcher + val removeMsg = RemoveBroadcast(broadcastId, removeFromDriver) + val requiredBlockManagers = blockManagerInfo.values.filter { info => + removeFromDriver || info.blockManagerId.executorId != "" + } + Future.sequence( + requiredBlockManagers.map { bm => + bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] + }.toSeq + ) } private def removeBlockManager(blockManagerId: BlockManagerId) { @@ -225,6 +269,61 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus }.toArray } + /** + * Return the block's status for all block managers, if any. NOTE: This is a + * potentially expensive operation and should only be used for testing. + * + * If askSlaves is true, the master queries each block manager for the most updated block + * statuses. This is useful when the master is not informed of the given block by all block + * managers. + */ + private def blockStatus( + blockId: BlockId, + askSlaves: Boolean): Map[BlockManagerId, Future[Option[BlockStatus]]] = { + import context.dispatcher + val getBlockStatus = GetBlockStatus(blockId) + /* + * Rather than blocking on the block status query, master actor should simply return + * Futures to avoid potential deadlocks. This can arise if there exists a block manager + * that is also waiting for this master actor's response to a previous message. + */ + blockManagerInfo.values.map { info => + val blockStatusFuture = + if (askSlaves) { + info.slaveActor.ask(getBlockStatus)(akkaTimeout).mapTo[Option[BlockStatus]] + } else { + Future { info.getStatus(blockId) } + } + (info.blockManagerId, blockStatusFuture) + }.toMap + } + + /** + * Return the ids of blocks present in all the block managers that match the given filter. + * NOTE: This is a potentially expensive operation and should only be used for testing. + * + * If askSlaves is true, the master queries each block manager for the most updated block + * statuses. This is useful when the master is not informed of the given block by all block + * managers. + */ + private def getMatchingBlockIds( + filter: BlockId => Boolean, + askSlaves: Boolean): Future[Seq[BlockId]] = { + import context.dispatcher + val getMatchingBlockIds = GetMatchingBlockIds(filter) + Future.sequence( + blockManagerInfo.values.map { info => + val future = + if (askSlaves) { + info.slaveActor.ask(getMatchingBlockIds)(akkaTimeout).mapTo[Seq[BlockId]] + } else { + Future { info.blocks.keys.filter(filter).toSeq } + } + future + } + ).map(_.flatten.toSeq) + } + private def register(id: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { if (!blockManagerInfo.contains(id)) { blockManagerIdByExecutor.get(id.executorId) match { @@ -334,6 +433,8 @@ private[spark] class BlockManagerInfo( logInfo("Registering block manager %s with %s RAM".format( blockManagerId.hostPort, Utils.bytesToString(maxMem))) + def getStatus(blockId: BlockId) = Option(_blocks.get(blockId)) + def updateLastSeenMs() { _lastSeenMs = System.currentTimeMillis() } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 8a36b5cc42dfd..2b53bf33b5fba 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -34,6 +34,13 @@ private[storage] object BlockManagerMessages { // Remove all blocks belonging to a specific RDD. case class RemoveRdd(rddId: Int) extends ToBlockManagerSlave + // Remove all blocks belonging to a specific shuffle. + case class RemoveShuffle(shuffleId: Int) extends ToBlockManagerSlave + + // Remove all blocks belonging to a specific broadcast. + case class RemoveBroadcast(broadcastId: Long, removeFromDriver: Boolean = true) + extends ToBlockManagerSlave + ////////////////////////////////////////////////////////////////////////////////// // Messages from slaves to the master. @@ -80,7 +87,8 @@ private[storage] object BlockManagerMessages { } object UpdateBlockInfo { - def apply(blockManagerId: BlockManagerId, + def apply( + blockManagerId: BlockManagerId, blockId: BlockId, storageLevel: StorageLevel, memSize: Long, @@ -108,7 +116,13 @@ private[storage] object BlockManagerMessages { case object GetMemoryStatus extends ToBlockManagerMaster - case object ExpireDeadHosts extends ToBlockManagerMaster - case object GetStorageStatus extends ToBlockManagerMaster + + case class GetBlockStatus(blockId: BlockId, askSlaves: Boolean = true) + extends ToBlockManagerMaster + + case class GetMatchingBlockIds(filter: BlockId => Boolean, askSlaves: Boolean = true) + extends ToBlockManagerMaster + + case object ExpireDeadHosts extends ToBlockManagerMaster } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala index bcfb82d3c7336..6d4db064dff58 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala @@ -17,8 +17,11 @@ package org.apache.spark.storage -import akka.actor.Actor +import scala.concurrent.Future +import akka.actor.{ActorRef, Actor} + +import org.apache.spark.{Logging, MapOutputTracker} import org.apache.spark.storage.BlockManagerMessages._ /** @@ -26,14 +29,59 @@ import org.apache.spark.storage.BlockManagerMessages._ * this is used to remove blocks from the slave's BlockManager. */ private[storage] -class BlockManagerSlaveActor(blockManager: BlockManager) extends Actor { - override def receive = { +class BlockManagerSlaveActor( + blockManager: BlockManager, + mapOutputTracker: MapOutputTracker) + extends Actor with Logging { + + import context.dispatcher + // Operations that involve removing blocks may be slow and should be done asynchronously + override def receive = { case RemoveBlock(blockId) => - blockManager.removeBlock(blockId) + doAsync[Boolean]("removing block " + blockId, sender) { + blockManager.removeBlock(blockId) + true + } case RemoveRdd(rddId) => - val numBlocksRemoved = blockManager.removeRdd(rddId) - sender ! numBlocksRemoved + doAsync[Int]("removing RDD " + rddId, sender) { + blockManager.removeRdd(rddId) + } + + case RemoveShuffle(shuffleId) => + doAsync[Boolean]("removing shuffle " + shuffleId, sender) { + if (mapOutputTracker != null) { + mapOutputTracker.unregisterShuffle(shuffleId) + } + blockManager.shuffleBlockManager.removeShuffle(shuffleId) + } + + case RemoveBroadcast(broadcastId, tellMaster) => + doAsync[Int]("removing broadcast " + broadcastId, sender) { + blockManager.removeBroadcast(broadcastId, tellMaster) + } + + case GetBlockStatus(blockId, _) => + sender ! blockManager.getStatus(blockId) + + case GetMatchingBlockIds(filter, _) => + sender ! blockManager.getMatchingBlockIds(filter) + } + + private def doAsync[T](actionMessage: String, responseActor: ActorRef)(body: => T) { + val future = Future { + logDebug(actionMessage) + body + } + future.onSuccess { case response => + logDebug("Done " + actionMessage + ", response is " + response) + responseActor ! response + logDebug("Sent response: " + response + " to " + responseActor) + } + future.onFailure { case t: Throwable => + logError("Error in " + actionMessage, t) + responseActor ! null.asInstanceOf[T] + } } } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index f3e1c38744d78..7a24c8f57f43b 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -90,6 +90,20 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD def getFile(blockId: BlockId): File = getFile(blockId.name) + /** Check if disk block manager has a block. */ + def containsBlock(blockId: BlockId): Boolean = { + getBlockLocation(blockId).file.exists() + } + + /** List all the blocks currently stored on disk by the disk manager. */ + def getAllBlocks(): Seq[BlockId] = { + // Get all the files inside the array of array of directories + subDirs.flatten.filter(_ != null).flatMap { dir => + val files = dir.list() + if (files != null) files else Seq.empty + }.map(BlockId.apply) + } + /** Produces a unique block id and File suitable for intermediate results. */ def createTempBlock(): (TempBlockId, File) = { var blockId = new TempBlockId(UUID.randomUUID()) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index bb07c8cb134cc..4cd4cdbd9909d 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -169,23 +169,43 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { throw new IllegalStateException("Failed to find shuffle block: " + id) } + /** Remove all the blocks / files and metadata related to a particular shuffle. */ + def removeShuffle(shuffleId: ShuffleId): Boolean = { + // Do not change the ordering of this, if shuffleStates should be removed only + // after the corresponding shuffle blocks have been removed + val cleaned = removeShuffleBlocks(shuffleId) + shuffleStates.remove(shuffleId) + cleaned + } + + /** Remove all the blocks / files related to a particular shuffle. */ + private def removeShuffleBlocks(shuffleId: ShuffleId): Boolean = { + shuffleStates.get(shuffleId) match { + case Some(state) => + if (consolidateShuffleFiles) { + for (fileGroup <- state.allFileGroups; file <- fileGroup.files) { + file.delete() + } + } else { + for (mapId <- state.completedMapTasks; reduceId <- 0 until state.numBuckets) { + val blockId = new ShuffleBlockId(shuffleId, mapId, reduceId) + blockManager.diskBlockManager.getFile(blockId).delete() + } + } + logInfo("Deleted all files for shuffle " + shuffleId) + true + case None => + logInfo("Could not find files for shuffle " + shuffleId + " for deleting") + false + } + } + private def physicalFileName(shuffleId: Int, bucketId: Int, fileId: Int) = { "merged_shuffle_%d_%d_%d".format(shuffleId, bucketId, fileId) } private def cleanup(cleanupTime: Long) { - shuffleStates.clearOldValues(cleanupTime, (shuffleId, state) => { - if (consolidateShuffleFiles) { - for (fileGroup <- state.allFileGroups; file <- fileGroup.files) { - file.delete() - } - } else { - for (mapId <- state.completedMapTasks; reduceId <- 0 until state.numBuckets) { - val blockId = new ShuffleBlockId(shuffleId, mapId, reduceId) - blockManager.diskBlockManager.getFile(blockId).delete() - } - } - }) + shuffleStates.clearOldValues(cleanupTime, (shuffleId, state) => removeShuffleBlocks(shuffleId)) } } diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index 226ed2a132b00..a107c5182b3be 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -22,7 +22,7 @@ import java.util.concurrent.ArrayBlockingQueue import akka.actor._ import util.Random -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.KryoSerializer @@ -48,7 +48,7 @@ private[spark] object ThreadingTest { val block = (1 to blockSize).map(_ => Random.nextInt()) val level = randomLevel() val startTime = System.currentTimeMillis() - manager.put(blockId, block.iterator, level, true) + manager.put(blockId, block.iterator, level, tellMaster = true) println("Pushed block " + blockId + " in " + (System.currentTimeMillis - startTime) + " ms") queue.add((blockId, block)) } @@ -101,7 +101,7 @@ private[spark] object ThreadingTest { conf) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf, - new SecurityManager(conf)) + new SecurityManager(conf), new MapOutputTrackerMaster(conf)) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) producers.foreach(_.start) diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 0448919e09161..7ebed5105b9fd 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -62,8 +62,8 @@ private[spark] class MetadataCleaner( private[spark] object MetadataCleanerType extends Enumeration { - val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, DAG_SCHEDULER, RESULT_TASK, - SHUFFLE_MAP_TASK, BLOCK_MANAGER, SHUFFLE_BLOCK_MANAGER, BROADCAST_VARS = Value + val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, BLOCK_MANAGER, + SHUFFLE_BLOCK_MANAGER, BROADCAST_VARS = Value type MetadataCleanerType = Value @@ -78,15 +78,16 @@ private[spark] object MetadataCleaner { conf.getInt("spark.cleaner.ttl", -1) } - def getDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType): Int = - { - conf.get(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString) - .toInt + def getDelaySeconds( + conf: SparkConf, + cleanerType: MetadataCleanerType.MetadataCleanerType): Int = { + conf.get(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString).toInt } - def setDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType, - delay: Int) - { + def setDelaySeconds( + conf: SparkConf, + cleanerType: MetadataCleanerType.MetadataCleanerType, + delay: Int) { conf.set(MetadataCleanerType.systemProperty(cleanerType), delay.toString) } diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala index ddbd084ed7f01..8de75ba9a9c92 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala @@ -17,48 +17,54 @@ package org.apache.spark.util +import java.util.Set +import java.util.Map.Entry import java.util.concurrent.ConcurrentHashMap -import scala.collection.JavaConversions -import scala.collection.immutable -import scala.collection.mutable.Map +import scala.collection.{JavaConversions, mutable} import org.apache.spark.Logging +private[spark] case class TimeStampedValue[V](value: V, timestamp: Long) + /** * This is a custom implementation of scala.collection.mutable.Map which stores the insertion * timestamp along with each key-value pair. If specified, the timestamp of each pair can be * updated every time it is accessed. Key-value pairs whose timestamp are older than a particular * threshold time can then be removed using the clearOldValues method. This is intended to * be a drop-in replacement of scala.collection.mutable.HashMap. - * @param updateTimeStampOnGet When enabled, the timestamp of a pair will be - * updated when it is accessed + * + * @param updateTimeStampOnGet Whether timestamp of a pair will be updated when it is accessed */ -class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) - extends Map[A, B]() with Logging { - val internalMap = new ConcurrentHashMap[A, (B, Long)]() +private[spark] class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) + extends mutable.Map[A, B]() with Logging { + + private val internalMap = new ConcurrentHashMap[A, TimeStampedValue[B]]() def get(key: A): Option[B] = { val value = internalMap.get(key) if (value != null && updateTimeStampOnGet) { - internalMap.replace(key, value, (value._1, currentTime)) + internalMap.replace(key, value, TimeStampedValue(value.value, currentTime)) } - Option(value).map(_._1) + Option(value).map(_.value) } def iterator: Iterator[(A, B)] = { - val jIterator = internalMap.entrySet().iterator() - JavaConversions.asScalaIterator(jIterator).map(kv => (kv.getKey, kv.getValue._1)) + val jIterator = getEntrySet.iterator + JavaConversions.asScalaIterator(jIterator).map(kv => (kv.getKey, kv.getValue.value)) } - override def + [B1 >: B](kv: (A, B1)): Map[A, B1] = { + def getEntrySet: Set[Entry[A, TimeStampedValue[B]]] = internalMap.entrySet + + override def + [B1 >: B](kv: (A, B1)): mutable.Map[A, B1] = { val newMap = new TimeStampedHashMap[A, B1] - newMap.internalMap.putAll(this.internalMap) - newMap.internalMap.put(kv._1, (kv._2, currentTime)) + val oldInternalMap = this.internalMap.asInstanceOf[ConcurrentHashMap[A, TimeStampedValue[B1]]] + newMap.internalMap.putAll(oldInternalMap) + kv match { case (a, b) => newMap.internalMap.put(a, TimeStampedValue(b, currentTime)) } newMap } - override def - (key: A): Map[A, B] = { + override def - (key: A): mutable.Map[A, B] = { val newMap = new TimeStampedHashMap[A, B] newMap.internalMap.putAll(this.internalMap) newMap.internalMap.remove(key) @@ -66,17 +72,10 @@ class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) } override def += (kv: (A, B)): this.type = { - internalMap.put(kv._1, (kv._2, currentTime)) + kv match { case (a, b) => internalMap.put(a, TimeStampedValue(b, currentTime)) } this } - // Should we return previous value directly or as Option ? - def putIfAbsent(key: A, value: B): Option[B] = { - val prev = internalMap.putIfAbsent(key, (value, currentTime)) - if (prev != null) Some(prev._1) else None - } - - override def -= (key: A): this.type = { internalMap.remove(key) this @@ -87,53 +86,65 @@ class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) } override def apply(key: A): B = { - val value = internalMap.get(key) - if (value == null) throw new NoSuchElementException() - value._1 + get(key).getOrElse { throw new NoSuchElementException() } } - override def filter(p: ((A, B)) => Boolean): Map[A, B] = { - JavaConversions.mapAsScalaConcurrentMap(internalMap).map(kv => (kv._1, kv._2._1)).filter(p) + override def filter(p: ((A, B)) => Boolean): mutable.Map[A, B] = { + JavaConversions.mapAsScalaConcurrentMap(internalMap) + .map { case (k, TimeStampedValue(v, t)) => (k, v) } + .filter(p) } - override def empty: Map[A, B] = new TimeStampedHashMap[A, B]() + override def empty: mutable.Map[A, B] = new TimeStampedHashMap[A, B]() override def size: Int = internalMap.size override def foreach[U](f: ((A, B)) => U) { - val iterator = internalMap.entrySet().iterator() - while(iterator.hasNext) { - val entry = iterator.next() - val kv = (entry.getKey, entry.getValue._1) + val it = getEntrySet.iterator + while(it.hasNext) { + val entry = it.next() + val kv = (entry.getKey, entry.getValue.value) f(kv) } } - def toMap: immutable.Map[A, B] = iterator.toMap + def putIfAbsent(key: A, value: B): Option[B] = { + val prev = internalMap.putIfAbsent(key, TimeStampedValue(value, currentTime)) + Option(prev).map(_.value) + } + + def putAll(map: Map[A, B]) { + map.foreach { case (k, v) => update(k, v) } + } + + def toMap: Map[A, B] = iterator.toMap - /** - * Removes old key-value pairs that have timestamp earlier than `threshTime`, - * calling the supplied function on each such entry before removing. - */ def clearOldValues(threshTime: Long, f: (A, B) => Unit) { - val iterator = internalMap.entrySet().iterator() - while (iterator.hasNext) { - val entry = iterator.next() - if (entry.getValue._2 < threshTime) { - f(entry.getKey, entry.getValue._1) + val it = getEntrySet.iterator + while (it.hasNext) { + val entry = it.next() + if (entry.getValue.timestamp < threshTime) { + f(entry.getKey, entry.getValue.value) logDebug("Removing key " + entry.getKey) - iterator.remove() + it.remove() } } } - /** - * Removes old key-value pairs that have timestamp earlier than `threshTime` - */ + /** Removes old key-value pairs that have timestamp earlier than `threshTime`. */ def clearOldValues(threshTime: Long) { clearOldValues(threshTime, (_, _) => ()) } - private def currentTime: Long = System.currentTimeMillis() + private def currentTime: Long = System.currentTimeMillis + // For testing + + def getTimeStampedValue(key: A): Option[TimeStampedValue[B]] = { + Option(internalMap.get(key)) + } + + def getTimestamp(key: A): Option[Long] = { + getTimeStampedValue(key).map(_.timestamp) + } } diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala new file mode 100644 index 0000000000000..b65017d6806c6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.lang.ref.WeakReference +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.mutable + +import org.apache.spark.Logging + +/** + * A wrapper of TimeStampedHashMap that ensures the values are weakly referenced and timestamped. + * + * If the value is garbage collected and the weak reference is null, get() will return a + * non-existent value. These entries are removed from the map periodically (every N inserts), as + * their values are no longer strongly reachable. Further, key-value pairs whose timestamps are + * older than a particular threshold can be removed using the clearOldValues method. + * + * TimeStampedWeakValueHashMap exposes a scala.collection.mutable.Map interface, which allows it + * to be a drop-in replacement for Scala HashMaps. Internally, it uses a Java ConcurrentHashMap, + * so all operations on this HashMap are thread-safe. + * + * @param updateTimeStampOnGet Whether timestamp of a pair will be updated when it is accessed. + */ +private[spark] class TimeStampedWeakValueHashMap[A, B](updateTimeStampOnGet: Boolean = false) + extends mutable.Map[A, B]() with Logging { + + import TimeStampedWeakValueHashMap._ + + private val internalMap = new TimeStampedHashMap[A, WeakReference[B]](updateTimeStampOnGet) + private val insertCount = new AtomicInteger(0) + + /** Return a map consisting only of entries whose values are still strongly reachable. */ + private def nonNullReferenceMap = internalMap.filter { case (_, ref) => ref.get != null } + + def get(key: A): Option[B] = internalMap.get(key) + + def iterator: Iterator[(A, B)] = nonNullReferenceMap.iterator + + override def + [B1 >: B](kv: (A, B1)): mutable.Map[A, B1] = { + val newMap = new TimeStampedWeakValueHashMap[A, B1] + val oldMap = nonNullReferenceMap.asInstanceOf[mutable.Map[A, WeakReference[B1]]] + newMap.internalMap.putAll(oldMap.toMap) + newMap.internalMap += kv + newMap + } + + override def - (key: A): mutable.Map[A, B] = { + val newMap = new TimeStampedWeakValueHashMap[A, B] + newMap.internalMap.putAll(nonNullReferenceMap.toMap) + newMap.internalMap -= key + newMap + } + + override def += (kv: (A, B)): this.type = { + internalMap += kv + if (insertCount.incrementAndGet() % CLEAR_NULL_VALUES_INTERVAL == 0) { + clearNullValues() + } + this + } + + override def -= (key: A): this.type = { + internalMap -= key + this + } + + override def update(key: A, value: B) = this += ((key, value)) + + override def apply(key: A): B = internalMap.apply(key) + + override def filter(p: ((A, B)) => Boolean): mutable.Map[A, B] = nonNullReferenceMap.filter(p) + + override def empty: mutable.Map[A, B] = new TimeStampedWeakValueHashMap[A, B]() + + override def size: Int = internalMap.size + + override def foreach[U](f: ((A, B)) => U) = nonNullReferenceMap.foreach(f) + + def putIfAbsent(key: A, value: B): Option[B] = internalMap.putIfAbsent(key, value) + + def toMap: Map[A, B] = iterator.toMap + + /** Remove old key-value pairs with timestamps earlier than `threshTime`. */ + def clearOldValues(threshTime: Long) = internalMap.clearOldValues(threshTime) + + /** Remove entries with values that are no longer strongly reachable. */ + def clearNullValues() { + val it = internalMap.getEntrySet.iterator + while (it.hasNext) { + val entry = it.next() + if (entry.getValue.value.get == null) { + logDebug("Removing key " + entry.getKey + " because it is no longer strongly reachable.") + it.remove() + } + } + } + + // For testing + + def getTimestamp(key: A): Option[Long] = { + internalMap.getTimeStampedValue(key).map(_.timestamp) + } + + def getReference(key: A): Option[WeakReference[B]] = { + internalMap.getTimeStampedValue(key).map(_.value) + } +} + +/** + * Helper methods for converting to and from WeakReferences. + */ +private object TimeStampedWeakValueHashMap { + + // Number of inserts after which entries with null references are removed + val CLEAR_NULL_VALUES_INTERVAL = 100 + + /* Implicit conversion methods to WeakReferences. */ + + implicit def toWeakReference[V](v: V): WeakReference[V] = new WeakReference[V](v) + + implicit def toWeakReferenceTuple[K, V](kv: (K, V)): (K, WeakReference[V]) = { + kv match { case (k, v) => (k, toWeakReference(v)) } + } + + implicit def toWeakReferenceFunction[K, V, R](p: ((K, V)) => R): ((K, WeakReference[V])) => R = { + (kv: (K, WeakReference[V])) => p(kv) + } + + /* Implicit conversion methods from WeakReferences. */ + + implicit def fromWeakReference[V](ref: WeakReference[V]): V = ref.get + + implicit def fromWeakReferenceOption[V](v: Option[WeakReference[V]]): Option[V] = { + v match { + case Some(ref) => Option(fromWeakReference(ref)) + case None => None + } + } + + implicit def fromWeakReferenceTuple[K, V](kv: (K, WeakReference[V])): (K, V) = { + kv match { case (k, v) => (k, fromWeakReference(v)) } + } + + implicit def fromWeakReferenceIterator[K, V]( + it: Iterator[(K, WeakReference[V])]): Iterator[(K, V)] = { + it.map(fromWeakReferenceTuple) + } + + implicit def fromWeakReferenceMap[K, V]( + map: mutable.Map[K, WeakReference[V]]) : mutable.Map[K, V] = { + mutable.Map(map.mapValues(fromWeakReference).toSeq: _*) + } +} diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 4435b21a7505e..59da51f3e0297 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -499,10 +499,10 @@ private[spark] object Utils extends Logging { private val hostPortParseResults = new ConcurrentHashMap[String, (String, Int)]() def parseHostPort(hostPort: String): (String, Int) = { - { - // Check cache first. - val cached = hostPortParseResults.get(hostPort) - if (cached != null) return cached + // Check cache first. + val cached = hostPortParseResults.get(hostPort) + if (cached != null) { + return cached } val indx: Int = hostPort.lastIndexOf(':') diff --git a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala index d2e303d81c4c8..c5f24c66ce0c1 100644 --- a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala @@ -56,7 +56,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf, securityManager = securityManagerBad) - val slaveTracker = new MapOutputTracker(conf) + val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) @@ -93,7 +93,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = badconf, securityManager = securityManagerBad) - val slaveTracker = new MapOutputTracker(conf) + val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) @@ -147,7 +147,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = goodconf, securityManager = securityManagerGood) - val slaveTracker = new MapOutputTracker(conf) + val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) @@ -200,7 +200,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = badconf, securityManager = securityManagerBad) - val slaveTracker = new MapOutputTracker(conf) + val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) diff --git a/core/src/test/scala/org/apache/spark/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/BroadcastSuite.scala index 96ba3929c1685..c9936256a5b95 100644 --- a/core/src/test/scala/org/apache/spark/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/BroadcastSuite.scala @@ -19,68 +19,297 @@ package org.apache.spark import org.scalatest.FunSuite -class BroadcastSuite extends FunSuite with LocalSparkContext { +import org.apache.spark.storage._ +import org.apache.spark.broadcast.{Broadcast, HttpBroadcast} +import org.apache.spark.storage.BroadcastBlockId +class BroadcastSuite extends FunSuite with LocalSparkContext { - override def afterEach() { - super.afterEach() - System.clearProperty("spark.broadcast.factory") - } + private val httpConf = broadcastConf("HttpBroadcastFactory") + private val torrentConf = broadcastConf("TorrentBroadcastFactory") test("Using HttpBroadcast locally") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") - sc = new SparkContext("local", "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to 2).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === Set((1, 10), (2, 10))) + sc = new SparkContext("local", "test", httpConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to 2).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === Set((1, 10), (2, 10))) } test("Accessing HttpBroadcast variables from multiple threads") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") - sc = new SparkContext("local[10]", "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to 10).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === (1 to 10).map(x => (x, 10)).toSet) + sc = new SparkContext("local[10]", "test", httpConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to 10).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to 10).map(x => (x, 10)).toSet) } test("Accessing HttpBroadcast variables in a local cluster") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") val numSlaves = 4 - sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to numSlaves).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === (1 to numSlaves).map(x => (x, 10)).toSet) + sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", httpConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to numSlaves).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to numSlaves).map(x => (x, 10)).toSet) } test("Using TorrentBroadcast locally") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory") - sc = new SparkContext("local", "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to 2).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === Set((1, 10), (2, 10))) + sc = new SparkContext("local", "test", torrentConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to 2).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === Set((1, 10), (2, 10))) } test("Accessing TorrentBroadcast variables from multiple threads") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory") - sc = new SparkContext("local[10]", "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to 10).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === (1 to 10).map(x => (x, 10)).toSet) + sc = new SparkContext("local[10]", "test", torrentConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to 10).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to 10).map(x => (x, 10)).toSet) } test("Accessing TorrentBroadcast variables in a local cluster") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory") val numSlaves = 4 - sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to numSlaves).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === (1 to numSlaves).map(x => (x, 10)).toSet) + sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", torrentConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to numSlaves).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to numSlaves).map(x => (x, 10)).toSet) + } + + test("Unpersisting HttpBroadcast on executors only in local mode") { + testUnpersistHttpBroadcast(distributed = false, removeFromDriver = false) + } + + test("Unpersisting HttpBroadcast on executors and driver in local mode") { + testUnpersistHttpBroadcast(distributed = false, removeFromDriver = true) + } + + test("Unpersisting HttpBroadcast on executors only in distributed mode") { + testUnpersistHttpBroadcast(distributed = true, removeFromDriver = false) + } + + test("Unpersisting HttpBroadcast on executors and driver in distributed mode") { + testUnpersistHttpBroadcast(distributed = true, removeFromDriver = true) + } + + test("Unpersisting TorrentBroadcast on executors only in local mode") { + testUnpersistTorrentBroadcast(distributed = false, removeFromDriver = false) + } + + test("Unpersisting TorrentBroadcast on executors and driver in local mode") { + testUnpersistTorrentBroadcast(distributed = false, removeFromDriver = true) + } + + test("Unpersisting TorrentBroadcast on executors only in distributed mode") { + testUnpersistTorrentBroadcast(distributed = true, removeFromDriver = false) + } + + test("Unpersisting TorrentBroadcast on executors and driver in distributed mode") { + testUnpersistTorrentBroadcast(distributed = true, removeFromDriver = true) + } + /** + * Verify the persistence of state associated with an HttpBroadcast in either local mode or + * local-cluster mode (when distributed = true). + * + * This test creates a broadcast variable, uses it on all executors, and then unpersists it. + * In between each step, this test verifies that the broadcast blocks and the broadcast file + * are present only on the expected nodes. + */ + private def testUnpersistHttpBroadcast(distributed: Boolean, removeFromDriver: Boolean) { + val numSlaves = if (distributed) 2 else 0 + + def getBlockIds(id: Long) = Seq[BroadcastBlockId](BroadcastBlockId(id)) + + // Verify that the broadcast file is created, and blocks are persisted only on the driver + def afterCreation(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + assert(blockIds.size === 1) + val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + assert(statuses.size === 1) + statuses.head match { case (bm, status) => + assert(bm.executorId === "", "Block should only be on the driver") + assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) + assert(status.memSize > 0, "Block should be in memory store on the driver") + assert(status.diskSize === 0, "Block should not be in disk store on the driver") + } + if (distributed) { + // this file is only generated in distributed mode + assert(HttpBroadcast.getFile(blockIds.head.broadcastId).exists, "Broadcast file not found!") + } + } + + // Verify that blocks are persisted in both the executors and the driver + def afterUsingBroadcast(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + assert(blockIds.size === 1) + val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + assert(statuses.size === numSlaves + 1) + statuses.foreach { case (_, status) => + assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) + assert(status.memSize > 0, "Block should be in memory store") + assert(status.diskSize === 0, "Block should not be in disk store") + } + } + + // Verify that blocks are unpersisted on all executors, and on all nodes if removeFromDriver + // is true. In the latter case, also verify that the broadcast file is deleted on the driver. + def afterUnpersist(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + assert(blockIds.size === 1) + val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + val expectedNumBlocks = if (removeFromDriver) 0 else 1 + val possiblyNot = if (removeFromDriver) "" else " not" + assert(statuses.size === expectedNumBlocks, + "Block should%s be unpersisted on the driver".format(possiblyNot)) + if (distributed && removeFromDriver) { + // this file is only generated in distributed mode + assert(!HttpBroadcast.getFile(blockIds.head.broadcastId).exists, + "Broadcast file should%s be deleted".format(possiblyNot)) + } + } + + testUnpersistBroadcast(distributed, numSlaves, httpConf, getBlockIds, afterCreation, + afterUsingBroadcast, afterUnpersist, removeFromDriver) + } + + /** + * Verify the persistence of state associated with an TorrentBroadcast in a local-cluster. + * + * This test creates a broadcast variable, uses it on all executors, and then unpersists it. + * In between each step, this test verifies that the broadcast blocks are present only on the + * expected nodes. + */ + private def testUnpersistTorrentBroadcast(distributed: Boolean, removeFromDriver: Boolean) { + val numSlaves = if (distributed) 2 else 0 + + def getBlockIds(id: Long) = { + val broadcastBlockId = BroadcastBlockId(id) + val metaBlockId = BroadcastBlockId(id, "meta") + // Assume broadcast value is small enough to fit into 1 piece + val pieceBlockId = BroadcastBlockId(id, "piece0") + if (distributed) { + // the metadata and piece blocks are generated only in distributed mode + Seq[BroadcastBlockId](broadcastBlockId, metaBlockId, pieceBlockId) + } else { + Seq[BroadcastBlockId](broadcastBlockId) + } + } + + // Verify that blocks are persisted only on the driver + def afterCreation(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + blockIds.foreach { blockId => + val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + assert(statuses.size === 1) + statuses.head match { case (bm, status) => + assert(bm.executorId === "", "Block should only be on the driver") + assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) + assert(status.memSize > 0, "Block should be in memory store on the driver") + assert(status.diskSize === 0, "Block should not be in disk store on the driver") + } + } + } + + // Verify that blocks are persisted in both the executors and the driver + def afterUsingBroadcast(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + blockIds.foreach { blockId => + val statuses = bmm.getBlockStatus(blockId, askSlaves = true) + if (blockId.field == "meta") { + // Meta data is only on the driver + assert(statuses.size === 1) + statuses.head match { case (bm, _) => assert(bm.executorId === "") } + } else { + // Other blocks are on both the executors and the driver + assert(statuses.size === numSlaves + 1, + blockId + " has " + statuses.size + " statuses: " + statuses.mkString(",")) + statuses.foreach { case (_, status) => + assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) + assert(status.memSize > 0, "Block should be in memory store") + assert(status.diskSize === 0, "Block should not be in disk store") + } + } + } + } + + // Verify that blocks are unpersisted on all executors, and on all nodes if removeFromDriver + // is true. + def afterUnpersist(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + val expectedNumBlocks = if (removeFromDriver) 0 else 1 + val possiblyNot = if (removeFromDriver) "" else " not" + blockIds.foreach { blockId => + val statuses = bmm.getBlockStatus(blockId, askSlaves = true) + assert(statuses.size === expectedNumBlocks, + "Block should%s be unpersisted on the driver".format(possiblyNot)) + } + } + + testUnpersistBroadcast(distributed, numSlaves, torrentConf, getBlockIds, afterCreation, + afterUsingBroadcast, afterUnpersist, removeFromDriver) + } + + /** + * This test runs in 4 steps: + * + * 1) Create broadcast variable, and verify that all state is persisted on the driver. + * 2) Use the broadcast variable on all executors, and verify that all state is persisted + * on both the driver and the executors. + * 3) Unpersist the broadcast, and verify that all state is removed where they should be. + * 4) [Optional] If removeFromDriver is false, we verify that the broadcast is re-usable. + */ + private def testUnpersistBroadcast( + distributed: Boolean, + numSlaves: Int, // used only when distributed = true + broadcastConf: SparkConf, + getBlockIds: Long => Seq[BroadcastBlockId], + afterCreation: (Seq[BroadcastBlockId], BlockManagerMaster) => Unit, + afterUsingBroadcast: (Seq[BroadcastBlockId], BlockManagerMaster) => Unit, + afterUnpersist: (Seq[BroadcastBlockId], BlockManagerMaster) => Unit, + removeFromDriver: Boolean) { + + sc = if (distributed) { + new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", broadcastConf) + } else { + new SparkContext("local", "test", broadcastConf) + } + val blockManagerMaster = sc.env.blockManager.master + val list = List[Int](1, 2, 3, 4) + + // Create broadcast variable + val broadcast = sc.broadcast(list) + val blocks = getBlockIds(broadcast.id) + afterCreation(blocks, blockManagerMaster) + + // Use broadcast variable on all executors + val partitions = 10 + assert(partitions > numSlaves) + val results = sc.parallelize(1 to partitions, partitions).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to partitions).map(x => (x, list.sum)).toSet) + afterUsingBroadcast(blocks, blockManagerMaster) + + // Unpersist broadcast + if (removeFromDriver) { + broadcast.destroy(blocking = true) + } else { + broadcast.unpersist(blocking = true) + } + afterUnpersist(blocks, blockManagerMaster) + + // If the broadcast is removed from driver, all subsequent uses of the broadcast variable + // should throw SparkExceptions. Otherwise, the result should be the same as before. + if (removeFromDriver) { + // Using this variable on the executors crashes them, which hangs the test. + // Instead, crash the driver by directly accessing the broadcast value. + intercept[SparkException] { broadcast.value } + intercept[SparkException] { broadcast.unpersist() } + intercept[SparkException] { broadcast.destroy(blocking = true) } + } else { + val results = sc.parallelize(1 to partitions, partitions).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to partitions).map(x => (x, list.sum)).toSet) + } } + /** Helper method to create a SparkConf that uses the given broadcast factory. */ + private def broadcastConf(factoryName: String): SparkConf = { + val conf = new SparkConf + conf.set("spark.broadcast.factory", "org.apache.spark.broadcast.%s".format(factoryName)) + conf + } } diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala new file mode 100644 index 0000000000000..e50981cf6fb20 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -0,0 +1,415 @@ +/* + * 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 + +import java.lang.ref.WeakReference + +import scala.collection.mutable.{HashSet, SynchronizedSet} +import scala.util.Random + +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.concurrent.Eventually +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.{BlockId, BroadcastBlockId, RDDBlockId, ShuffleBlockId} + +class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { + + implicit val defaultTimeout = timeout(10000 millis) + val conf = new SparkConf() + .setMaster("local[2]") + .setAppName("ContextCleanerSuite") + .set("spark.cleaner.referenceTracking.blocking", "true") + + before { + sc = new SparkContext(conf) + } + + after { + if (sc != null) { + sc.stop() + sc = null + } + } + + + test("cleanup RDD") { + val rdd = newRDD.persist() + val collected = rdd.collect().toList + val tester = new CleanerTester(sc, rddIds = Seq(rdd.id)) + + // Explicit cleanup + cleaner.doCleanupRDD(rdd.id, blocking = true) + tester.assertCleanup() + + // Verify that RDDs can be re-executed after cleaning up + assert(rdd.collect().toList === collected) + } + + test("cleanup shuffle") { + val (rdd, shuffleDeps) = newRDDWithShuffleDependencies + val collected = rdd.collect().toList + val tester = new CleanerTester(sc, shuffleIds = shuffleDeps.map(_.shuffleId)) + + // Explicit cleanup + shuffleDeps.foreach(s => cleaner.doCleanupShuffle(s.shuffleId, blocking = true)) + tester.assertCleanup() + + // Verify that shuffles can be re-executed after cleaning up + assert(rdd.collect().toList === collected) + } + + test("cleanup broadcast") { + val broadcast = newBroadcast + val tester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) + + // Explicit cleanup + cleaner.doCleanupBroadcast(broadcast.id, blocking = true) + tester.assertCleanup() + } + + test("automatically cleanup RDD") { + var rdd = newRDD.persist() + rdd.count() + + // Test that GC does not cause RDD cleanup due to a strong reference + val preGCTester = new CleanerTester(sc, rddIds = Seq(rdd.id)) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC causes RDD cleanup after dereferencing the RDD + val postGCTester = new CleanerTester(sc, rddIds = Seq(rdd.id)) + rdd = null // Make RDD out of scope + runGC() + postGCTester.assertCleanup() + } + + test("automatically cleanup shuffle") { + var rdd = newShuffleRDD + rdd.count() + + // Test that GC does not cause shuffle cleanup due to a strong reference + val preGCTester = new CleanerTester(sc, shuffleIds = Seq(0)) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC causes shuffle cleanup after dereferencing the RDD + val postGCTester = new CleanerTester(sc, shuffleIds = Seq(0)) + rdd = null // Make RDD out of scope, so that corresponding shuffle goes out of scope + runGC() + postGCTester.assertCleanup() + } + + test("automatically cleanup broadcast") { + var broadcast = newBroadcast + + // Test that GC does not cause broadcast cleanup due to a strong reference + val preGCTester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC causes broadcast cleanup after dereferencing the broadcast variable + val postGCTester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) + broadcast = null // Make broadcast variable out of scope + runGC() + postGCTester.assertCleanup() + } + + test("automatically cleanup RDD + shuffle + broadcast") { + val numRdds = 100 + val numBroadcasts = 4 // Broadcasts are more costly + val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast).toBuffer + val rddIds = sc.persistentRdds.keys.toSeq + val shuffleIds = 0 until sc.newShuffleId + val broadcastIds = 0L until numBroadcasts + + val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC triggers the cleanup of all variables after the dereferencing them + val postGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + broadcastBuffer.clear() + rddBuffer.clear() + runGC() + postGCTester.assertCleanup() + } + + test("automatically cleanup RDD + shuffle + broadcast in distributed mode") { + sc.stop() + + val conf2 = new SparkConf() + .setMaster("local-cluster[2, 1, 512]") + .setAppName("ContextCleanerSuite") + .set("spark.cleaner.referenceTracking.blocking", "true") + sc = new SparkContext(conf2) + + val numRdds = 10 + val numBroadcasts = 4 // Broadcasts are more costly + val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast).toBuffer + val rddIds = sc.persistentRdds.keys.toSeq + val shuffleIds = 0 until sc.newShuffleId + val broadcastIds = 0L until numBroadcasts + + val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC triggers the cleanup of all variables after the dereferencing them + val postGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + broadcastBuffer.clear() + rddBuffer.clear() + runGC() + postGCTester.assertCleanup() + } + + //------ Helper functions ------ + + def newRDD = sc.makeRDD(1 to 10) + def newPairRDD = newRDD.map(_ -> 1) + def newShuffleRDD = newPairRDD.reduceByKey(_ + _) + def newBroadcast = sc.broadcast(1 to 100) + def newRDDWithShuffleDependencies: (RDD[_], Seq[ShuffleDependency[_, _]]) = { + def getAllDependencies(rdd: RDD[_]): Seq[Dependency[_]] = { + rdd.dependencies ++ rdd.dependencies.flatMap { dep => + getAllDependencies(dep.rdd) + } + } + val rdd = newShuffleRDD + + // Get all the shuffle dependencies + val shuffleDeps = getAllDependencies(rdd) + .filter(_.isInstanceOf[ShuffleDependency[_, _]]) + .map(_.asInstanceOf[ShuffleDependency[_, _]]) + (rdd, shuffleDeps) + } + + def randomRdd = { + val rdd: RDD[_] = Random.nextInt(3) match { + case 0 => newRDD + case 1 => newShuffleRDD + case 2 => newPairRDD.join(newPairRDD) + } + if (Random.nextBoolean()) rdd.persist() + rdd.count() + rdd + } + + def randomBroadcast = { + sc.broadcast(Random.nextInt(Int.MaxValue)) + } + + /** Run GC and make sure it actually has run */ + def runGC() { + val weakRef = new WeakReference(new Object()) + val startTime = System.currentTimeMillis + System.gc() // Make a best effort to run the garbage collection. It *usually* runs GC. + // Wait until a weak reference object has been GCed + while(System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { + System.gc() + Thread.sleep(200) + } + } + + def cleaner = sc.cleaner.get +} + + +/** Class to test whether RDDs, shuffles, etc. have been successfully cleaned. */ +class CleanerTester( + sc: SparkContext, + rddIds: Seq[Int] = Seq.empty, + shuffleIds: Seq[Int] = Seq.empty, + broadcastIds: Seq[Long] = Seq.empty) + extends Logging { + + val toBeCleanedRDDIds = new HashSet[Int] with SynchronizedSet[Int] ++= rddIds + val toBeCleanedShuffleIds = new HashSet[Int] with SynchronizedSet[Int] ++= shuffleIds + val toBeCleanedBroadcstIds = new HashSet[Long] with SynchronizedSet[Long] ++= broadcastIds + val isDistributed = !sc.isLocal + + val cleanerListener = new CleanerListener { + def rddCleaned(rddId: Int): Unit = { + toBeCleanedRDDIds -= rddId + logInfo("RDD "+ rddId + " cleaned") + } + + def shuffleCleaned(shuffleId: Int): Unit = { + toBeCleanedShuffleIds -= shuffleId + logInfo("Shuffle " + shuffleId + " cleaned") + } + + def broadcastCleaned(broadcastId: Long): Unit = { + toBeCleanedBroadcstIds -= broadcastId + logInfo("Broadcast" + broadcastId + " cleaned") + } + } + + val MAX_VALIDATION_ATTEMPTS = 10 + val VALIDATION_ATTEMPT_INTERVAL = 100 + + logInfo("Attempting to validate before cleanup:\n" + uncleanedResourcesToString) + preCleanupValidate() + sc.cleaner.get.attachListener(cleanerListener) + + /** Assert that all the stuff has been cleaned up */ + def assertCleanup()(implicit waitTimeout: Eventually.Timeout) { + try { + eventually(waitTimeout, interval(100 millis)) { + assert(isAllCleanedUp) + } + postCleanupValidate() + } finally { + logInfo("Resources left from cleaning up:\n" + uncleanedResourcesToString) + } + } + + /** Verify that RDDs, shuffles, etc. occupy resources */ + private def preCleanupValidate() { + assert(rddIds.nonEmpty || shuffleIds.nonEmpty || broadcastIds.nonEmpty, "Nothing to cleanup") + + // Verify the RDDs have been persisted and blocks are present + rddIds.foreach { rddId => + assert( + sc.persistentRdds.contains(rddId), + "RDD " + rddId + " have not been persisted, cannot start cleaner test" + ) + + assert( + !getRDDBlocks(rddId).isEmpty, + "Blocks of RDD " + rddId + " cannot be found in block manager, " + + "cannot start cleaner test" + ) + } + + // Verify the shuffle ids are registered and blocks are present + shuffleIds.foreach { shuffleId => + assert( + mapOutputTrackerMaster.containsShuffle(shuffleId), + "Shuffle " + shuffleId + " have not been registered, cannot start cleaner test" + ) + + assert( + !getShuffleBlocks(shuffleId).isEmpty, + "Blocks of shuffle " + shuffleId + " cannot be found in block manager, " + + "cannot start cleaner test" + ) + } + + // Verify that the broadcast blocks are present + broadcastIds.foreach { broadcastId => + assert( + !getBroadcastBlocks(broadcastId).isEmpty, + "Blocks of broadcast " + broadcastId + "cannot be found in block manager, " + + "cannot start cleaner test" + ) + } + } + + /** + * Verify that RDDs, shuffles, etc. do not occupy resources. Tests multiple times as there is + * as there is not guarantee on how long it will take clean up the resources. + */ + private def postCleanupValidate() { + // Verify the RDDs have been persisted and blocks are present + rddIds.foreach { rddId => + assert( + !sc.persistentRdds.contains(rddId), + "RDD " + rddId + " was not cleared from sc.persistentRdds" + ) + + assert( + getRDDBlocks(rddId).isEmpty, + "Blocks of RDD " + rddId + " were not cleared from block manager" + ) + } + + // Verify the shuffle ids are registered and blocks are present + shuffleIds.foreach { shuffleId => + assert( + !mapOutputTrackerMaster.containsShuffle(shuffleId), + "Shuffle " + shuffleId + " was not deregistered from map output tracker" + ) + + assert( + getShuffleBlocks(shuffleId).isEmpty, + "Blocks of shuffle " + shuffleId + " were not cleared from block manager" + ) + } + + // Verify that the broadcast blocks are present + broadcastIds.foreach { broadcastId => + assert( + getBroadcastBlocks(broadcastId).isEmpty, + "Blocks of broadcast " + broadcastId + " were not cleared from block manager" + ) + } + } + + private def uncleanedResourcesToString = { + s""" + |\tRDDs = ${toBeCleanedRDDIds.toSeq.sorted.mkString("[", ", ", "]")} + |\tShuffles = ${toBeCleanedShuffleIds.toSeq.sorted.mkString("[", ", ", "]")} + |\tBroadcasts = ${toBeCleanedBroadcstIds.toSeq.sorted.mkString("[", ", ", "]")} + """.stripMargin + } + + private def isAllCleanedUp = + toBeCleanedRDDIds.isEmpty && + toBeCleanedShuffleIds.isEmpty && + toBeCleanedBroadcstIds.isEmpty + + private def getRDDBlocks(rddId: Int): Seq[BlockId] = { + blockManager.master.getMatchingBlockIds( _ match { + case RDDBlockId(`rddId`, _) => true + case _ => false + }, askSlaves = true) + } + + private def getShuffleBlocks(shuffleId: Int): Seq[BlockId] = { + blockManager.master.getMatchingBlockIds( _ match { + case ShuffleBlockId(`shuffleId`, _, _) => true + case _ => false + }, askSlaves = true) + } + + private def getBroadcastBlocks(broadcastId: Long): Seq[BlockId] = { + blockManager.master.getMatchingBlockIds( _ match { + case BroadcastBlockId(`broadcastId`, _) => true + case _ => false + }, askSlaves = true) + } + + private def blockManager = sc.env.blockManager + private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] +} diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index a5bd72eb0a122..6b2571cd9295e 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -57,12 +57,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { tracker.stop() } - test("master register and fetch") { + test("master register shuffle and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) tracker.registerShuffle(10, 2) + assert(tracker.containsShuffle(10)) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) val size1000 = MapOutputTracker.decompressSize(compressedSize1000) @@ -77,7 +78,25 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { tracker.stop() } - test("master register and unregister and fetch") { + test("master register and unregister shuffle") { + val actorSystem = ActorSystem("test") + val tracker = new MapOutputTrackerMaster(conf) + tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) + tracker.registerShuffle(10, 2) + val compressedSize1000 = MapOutputTracker.compressSize(1000L) + val compressedSize10000 = MapOutputTracker.compressSize(10000L) + tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000, 0), + Array(compressedSize1000, compressedSize10000))) + tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000, 0), + Array(compressedSize10000, compressedSize1000))) + assert(tracker.containsShuffle(10)) + assert(tracker.getServerStatuses(10, 0).nonEmpty) + tracker.unregisterShuffle(10) + assert(!tracker.containsShuffle(10)) + assert(tracker.getServerStatuses(10, 0).isEmpty) + } + + test("master register shuffle and unregister map output and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = @@ -114,7 +133,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf, securityManager = new SecurityManager(conf)) - val slaveTracker = new MapOutputTracker(conf) + val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index b6dd0526105a0..e10ec7d2624a0 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} @@ -42,6 +42,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT var oldArch: String = null conf.set("spark.authenticate", "false") val securityMgr = new SecurityManager(conf) + val mapOutputTracker = new MapOutputTrackerMaster(conf) // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test conf.set("spark.kryoserializer.buffer.mb", "1") @@ -130,7 +131,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 1 manager interaction") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -160,9 +162,10 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 2 managers interaction") { - store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer(conf), 2000, conf, - securityMgr) + securityMgr, mapOutputTracker) val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -177,7 +180,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("removing block") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -225,7 +229,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("removing rdd") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -257,9 +262,82 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT master.getLocations(rdd(0, 1)) should have size 0 } + test("removing broadcast") { + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) + val driverStore = store + val executorStore = new BlockManager("executor", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) + val a1 = new Array[Byte](400) + val a2 = new Array[Byte](400) + val a3 = new Array[Byte](400) + val a4 = new Array[Byte](400) + + val broadcast0BlockId = BroadcastBlockId(0) + val broadcast1BlockId = BroadcastBlockId(1) + val broadcast2BlockId = BroadcastBlockId(2) + val broadcast2BlockId2 = BroadcastBlockId(2, "_") + + // insert broadcast blocks in both the stores + Seq(driverStore, executorStore).foreach { case s => + s.putSingle(broadcast0BlockId, a1, StorageLevel.DISK_ONLY) + s.putSingle(broadcast1BlockId, a2, StorageLevel.DISK_ONLY) + s.putSingle(broadcast2BlockId, a3, StorageLevel.DISK_ONLY) + s.putSingle(broadcast2BlockId2, a4, StorageLevel.DISK_ONLY) + } + + // verify whether the blocks exist in both the stores + Seq(driverStore, executorStore).foreach { case s => + s.getLocal(broadcast0BlockId) should not be (None) + s.getLocal(broadcast1BlockId) should not be (None) + s.getLocal(broadcast2BlockId) should not be (None) + s.getLocal(broadcast2BlockId2) should not be (None) + } + + // remove broadcast 0 block only from executors + master.removeBroadcast(0, removeFromMaster = false, blocking = true) + + // only broadcast 0 block should be removed from the executor store + executorStore.getLocal(broadcast0BlockId) should be (None) + executorStore.getLocal(broadcast1BlockId) should not be (None) + executorStore.getLocal(broadcast2BlockId) should not be (None) + + // nothing should be removed from the driver store + driverStore.getLocal(broadcast0BlockId) should not be (None) + driverStore.getLocal(broadcast1BlockId) should not be (None) + driverStore.getLocal(broadcast2BlockId) should not be (None) + + // remove broadcast 0 block from the driver as well + master.removeBroadcast(0, removeFromMaster = true, blocking = true) + driverStore.getLocal(broadcast0BlockId) should be (None) + driverStore.getLocal(broadcast1BlockId) should not be (None) + + // remove broadcast 1 block from both the stores asynchronously + // and verify all broadcast 1 blocks have been removed + master.removeBroadcast(1, removeFromMaster = true, blocking = false) + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + driverStore.getLocal(broadcast1BlockId) should be (None) + executorStore.getLocal(broadcast1BlockId) should be (None) + } + + // remove broadcast 2 from both the stores asynchronously + // and verify all broadcast 2 blocks have been removed + master.removeBroadcast(2, removeFromMaster = true, blocking = false) + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + driverStore.getLocal(broadcast2BlockId) should be (None) + driverStore.getLocal(broadcast2BlockId2) should be (None) + executorStore.getLocal(broadcast2BlockId) should be (None) + executorStore.getLocal(broadcast2BlockId2) should be (None) + } + executorStore.stop() + driverStore.stop() + store = null + } + test("reregistration on heart beat") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -275,7 +353,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("reregistration on block update") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) @@ -294,7 +373,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("reregistration doesn't dead lock") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = List(new Array[Byte](400)) @@ -331,7 +411,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -350,7 +431,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -369,7 +451,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of same RDD") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -388,7 +471,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of multiple RDDs") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 2), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) @@ -414,7 +498,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT // TODO Make the spark.test.tachyon.enable true after using tachyon 0.5.0 testing jar. val tachyonUnitTestEnabled = conf.getBoolean("spark.test.tachyon.enable", false) if (tachyonUnitTestEnabled) { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -430,7 +515,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("on-disk storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -443,7 +529,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -458,7 +545,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -473,7 +561,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -488,7 +577,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization and getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -503,7 +593,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -525,7 +616,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU with streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -549,7 +641,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels and streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -595,7 +688,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("overly large block") { - store = new BlockManager("", actorSystem, master, serializer, 500, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 500, conf, + securityMgr, mapOutputTracker) store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.getSingle("a1") === None, "a1 was in store") store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK) @@ -606,7 +700,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block compression") { try { conf.set("spark.shuffle.compress", "true") - store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") @@ -614,7 +709,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store = null conf.set("spark.shuffle.compress", "false") - store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 1000, "shuffle_0_0_0 was compressed") @@ -622,7 +718,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store = null conf.set("spark.broadcast.compress", "true") - store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 100, "broadcast_0 was not compressed") @@ -630,28 +727,32 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store = null conf.set("spark.broadcast.compress", "false") - store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 1000, "broadcast_0 was compressed") store.stop() store = null conf.set("spark.rdd.compress", "true") - store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) <= 100, "rdd_0_0 was not compressed") store.stop() store = null conf.set("spark.rdd.compress", "false") - store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) >= 1000, "rdd_0_0 was compressed") store.stop() store = null // Check that any other block types are also kept uncompressed - store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed") store.stop() @@ -666,7 +767,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block store put failure") { // Use Java serializer so we can create an unserializable error. store = new BlockManager("", actorSystem, master, new JavaSerializer(conf), 1200, conf, - securityMgr) + securityMgr, mapOutputTracker) // The put should fail since a1 is not serializable. class UnserializableClass @@ -682,7 +783,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("updated block statuses") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val list = List.fill(2)(new Array[Byte](200)) val bigList = List.fill(8)(new Array[Byte](200)) @@ -735,8 +837,83 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(!store.get("list5").isDefined, "list5 was in store") } + test("query block statuses") { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) + val list = List.fill(2)(new Array[Byte](200)) + + // Tell master. By LRU, only list2 and list3 remains. + store.put("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.put("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + + // getLocations and getBlockStatus should yield the same locations + assert(store.master.getLocations("list1").size === 0) + assert(store.master.getLocations("list2").size === 1) + assert(store.master.getLocations("list3").size === 1) + assert(store.master.getBlockStatus("list1", askSlaves = false).size === 0) + assert(store.master.getBlockStatus("list2", askSlaves = false).size === 1) + assert(store.master.getBlockStatus("list3", askSlaves = false).size === 1) + assert(store.master.getBlockStatus("list1", askSlaves = true).size === 0) + assert(store.master.getBlockStatus("list2", askSlaves = true).size === 1) + assert(store.master.getBlockStatus("list3", askSlaves = true).size === 1) + + // This time don't tell master and see what happens. By LRU, only list5 and list6 remains. + store.put("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + store.put("list5", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.put("list6", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + + // getLocations should return nothing because the master is not informed + // getBlockStatus without asking slaves should have the same result + // getBlockStatus with asking slaves, however, should return the actual block statuses + assert(store.master.getLocations("list4").size === 0) + assert(store.master.getLocations("list5").size === 0) + assert(store.master.getLocations("list6").size === 0) + assert(store.master.getBlockStatus("list4", askSlaves = false).size === 0) + assert(store.master.getBlockStatus("list5", askSlaves = false).size === 0) + assert(store.master.getBlockStatus("list6", askSlaves = false).size === 0) + assert(store.master.getBlockStatus("list4", askSlaves = true).size === 0) + assert(store.master.getBlockStatus("list5", askSlaves = true).size === 1) + assert(store.master.getBlockStatus("list6", askSlaves = true).size === 1) + } + + test("get matching blocks") { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) + val list = List.fill(2)(new Array[Byte](10)) + + // insert some blocks + store.put("list1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.put("list3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + + // getLocations and getBlockStatus should yield the same locations + assert(store.master.getMatchingBlockIds(_.toString.contains("list"), askSlaves = false).size === 3) + assert(store.master.getMatchingBlockIds(_.toString.contains("list1"), askSlaves = false).size === 1) + + // insert some more blocks + store.put("newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.put("newlist2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.put("newlist3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + + // getLocations and getBlockStatus should yield the same locations + assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = false).size === 1) + assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = true).size === 3) + + val blockIds = Seq(RDDBlockId(1, 0), RDDBlockId(1, 1), RDDBlockId(2, 0)) + blockIds.foreach { blockId => + store.put(blockId, list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + } + val matchedBlockIds = store.master.getMatchingBlockIds(_ match { + case RDDBlockId(1, _) => true + case _ => false + }, askSlaves = true) + assert(matchedBlockIds.toSet === Set(RDDBlockId(1, 0), RDDBlockId(1, 1))) + } + test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) store.putSingle(rdd(0, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) // Access rdd_1_0 to ensure it's not least recently used. diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 62f9b3cc7b2c1..808ddfdcf45d8 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -59,8 +59,16 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { val newFile = diskBlockManager.getFile(blockId) writeToFile(newFile, 10) assertSegmentEquals(blockId, blockId.name, 0, 10) - + assert(diskBlockManager.containsBlock(blockId)) newFile.delete() + assert(!diskBlockManager.containsBlock(blockId)) + } + + test("enumerating blocks") { + val ids = (1 to 100).map(i => TestBlockId("test_" + i)) + val files = ids.map(id => diskBlockManager.getFile(id)) + files.foreach(file => writeToFile(file, 10)) + assert(diskBlockManager.getAllBlocks.toSet === ids.toSet) } test("block appending") { diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 054eb01a64c11..7bab7da8fed68 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -108,8 +108,7 @@ class JsonProtocolSuite extends FunSuite { // BlockId testBlockId(RDDBlockId(1, 2)) testBlockId(ShuffleBlockId(1, 2, 3)) - testBlockId(BroadcastBlockId(1L)) - testBlockId(BroadcastHelperBlockId(BroadcastBlockId(2L), "Spark")) + testBlockId(BroadcastBlockId(1L, "insert_words_of_wisdom_here")) testBlockId(TaskResultBlockId(1L)) testBlockId(StreamBlockId(1, 2L)) } @@ -555,4 +554,4 @@ class JsonProtocolSuite extends FunSuite { {"Event":"SparkListenerUnpersistRDD","RDD ID":12345} """ - } +} diff --git a/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala new file mode 100644 index 0000000000000..6a5653ed2fb54 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala @@ -0,0 +1,264 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.lang.ref.WeakReference + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.util.Random + +import org.scalatest.FunSuite + +class TimeStampedHashMapSuite extends FunSuite { + + // Test the testMap function - a Scala HashMap should obviously pass + testMap(new mutable.HashMap[String, String]()) + + // Test TimeStampedHashMap basic functionality + testMap(new TimeStampedHashMap[String, String]()) + testMapThreadSafety(new TimeStampedHashMap[String, String]()) + + // Test TimeStampedWeakValueHashMap basic functionality + testMap(new TimeStampedWeakValueHashMap[String, String]()) + testMapThreadSafety(new TimeStampedWeakValueHashMap[String, String]()) + + test("TimeStampedHashMap - clearing by timestamp") { + // clearing by insertion time + val map = new TimeStampedHashMap[String, String](updateTimeStampOnGet = false) + map("k1") = "v1" + assert(map("k1") === "v1") + Thread.sleep(10) + val threshTime = System.currentTimeMillis + assert(map.getTimestamp("k1").isDefined) + assert(map.getTimestamp("k1").get < threshTime) + map.clearOldValues(threshTime) + assert(map.get("k1") === None) + + // clearing by modification time + val map1 = new TimeStampedHashMap[String, String](updateTimeStampOnGet = true) + map1("k1") = "v1" + map1("k2") = "v2" + assert(map1("k1") === "v1") + Thread.sleep(10) + val threshTime1 = System.currentTimeMillis + Thread.sleep(10) + assert(map1("k2") === "v2") // access k2 to update its access time to > threshTime + assert(map1.getTimestamp("k1").isDefined) + assert(map1.getTimestamp("k1").get < threshTime1) + assert(map1.getTimestamp("k2").isDefined) + assert(map1.getTimestamp("k2").get >= threshTime1) + map1.clearOldValues(threshTime1) //should only clear k1 + assert(map1.get("k1") === None) + assert(map1.get("k2").isDefined) + } + + test("TimeStampedWeakValueHashMap - clearing by timestamp") { + // clearing by insertion time + val map = new TimeStampedWeakValueHashMap[String, String](updateTimeStampOnGet = false) + map("k1") = "v1" + assert(map("k1") === "v1") + Thread.sleep(10) + val threshTime = System.currentTimeMillis + assert(map.getTimestamp("k1").isDefined) + assert(map.getTimestamp("k1").get < threshTime) + map.clearOldValues(threshTime) + assert(map.get("k1") === None) + + // clearing by modification time + val map1 = new TimeStampedWeakValueHashMap[String, String](updateTimeStampOnGet = true) + map1("k1") = "v1" + map1("k2") = "v2" + assert(map1("k1") === "v1") + Thread.sleep(10) + val threshTime1 = System.currentTimeMillis + Thread.sleep(10) + assert(map1("k2") === "v2") // access k2 to update its access time to > threshTime + assert(map1.getTimestamp("k1").isDefined) + assert(map1.getTimestamp("k1").get < threshTime1) + assert(map1.getTimestamp("k2").isDefined) + assert(map1.getTimestamp("k2").get >= threshTime1) + map1.clearOldValues(threshTime1) //should only clear k1 + assert(map1.get("k1") === None) + assert(map1.get("k2").isDefined) + } + + test("TimeStampedWeakValueHashMap - clearing weak references") { + var strongRef = new Object + val weakRef = new WeakReference(strongRef) + val map = new TimeStampedWeakValueHashMap[String, Object] + map("k1") = strongRef + map("k2") = "v2" + map("k3") = "v3" + assert(map("k1") === strongRef) + + // clear strong reference to "k1" + strongRef = null + val startTime = System.currentTimeMillis + System.gc() // Make a best effort to run the garbage collection. It *usually* runs GC. + System.runFinalization() // Make a best effort to call finalizer on all cleaned objects. + while(System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { + System.gc() + System.runFinalization() + Thread.sleep(100) + } + assert(map.getReference("k1").isDefined) + val ref = map.getReference("k1").get + assert(ref.get === null) + assert(map.get("k1") === None) + + // operations should only display non-null entries + assert(map.iterator.forall { case (k, v) => k != "k1" }) + assert(map.filter { case (k, v) => k != "k2" }.size === 1) + assert(map.filter { case (k, v) => k != "k2" }.head._1 === "k3") + assert(map.toMap.size === 2) + assert(map.toMap.forall { case (k, v) => k != "k1" }) + val buffer = new ArrayBuffer[String] + map.foreach { case (k, v) => buffer += v.toString } + assert(buffer.size === 2) + assert(buffer.forall(_ != "k1")) + val plusMap = map + (("k4", "v4")) + assert(plusMap.size === 3) + assert(plusMap.forall { case (k, v) => k != "k1" }) + val minusMap = map - "k2" + assert(minusMap.size === 1) + assert(minusMap.head._1 == "k3") + + // clear null values - should only clear k1 + map.clearNullValues() + assert(map.getReference("k1") === None) + assert(map.get("k1") === None) + assert(map.get("k2").isDefined) + assert(map.get("k2").get === "v2") + } + + /** Test basic operations of a Scala mutable Map. */ + def testMap(hashMapConstructor: => mutable.Map[String, String]) { + def newMap() = hashMapConstructor + val testMap1 = newMap() + val testMap2 = newMap() + val name = testMap1.getClass.getSimpleName + + test(name + " - basic test") { + // put, get, and apply + testMap1 += (("k1", "v1")) + assert(testMap1.get("k1").isDefined) + assert(testMap1.get("k1").get === "v1") + testMap1("k2") = "v2" + assert(testMap1.get("k2").isDefined) + assert(testMap1.get("k2").get === "v2") + assert(testMap1("k2") === "v2") + testMap1.update("k3", "v3") + assert(testMap1.get("k3").isDefined) + assert(testMap1.get("k3").get === "v3") + + // remove + testMap1.remove("k1") + assert(testMap1.get("k1").isEmpty) + testMap1.remove("k2") + intercept[NoSuchElementException] { + testMap1("k2") // Map.apply() causes exception + } + testMap1 -= "k3" + assert(testMap1.get("k3").isEmpty) + + // multi put + val keys = (1 to 100).map(_.toString) + val pairs = keys.map(x => (x, x * 2)) + assert((testMap2 ++ pairs).iterator.toSet === pairs.toSet) + testMap2 ++= pairs + + // iterator + assert(testMap2.iterator.toSet === pairs.toSet) + + // filter + val filtered = testMap2.filter { case (_, v) => v.toInt % 2 == 0 } + val evenPairs = pairs.filter { case (_, v) => v.toInt % 2 == 0 } + assert(filtered.iterator.toSet === evenPairs.toSet) + + // foreach + val buffer = new ArrayBuffer[(String, String)] + testMap2.foreach(x => buffer += x) + assert(testMap2.toSet === buffer.toSet) + + // multi remove + testMap2("k1") = "v1" + testMap2 --= keys + assert(testMap2.size === 1) + assert(testMap2.iterator.toSeq.head === ("k1", "v1")) + + // + + val testMap3 = testMap2 + (("k0", "v0")) + assert(testMap3.size === 2) + assert(testMap3.get("k1").isDefined) + assert(testMap3.get("k1").get === "v1") + assert(testMap3.get("k0").isDefined) + assert(testMap3.get("k0").get === "v0") + + // - + val testMap4 = testMap3 - "k0" + assert(testMap4.size === 1) + assert(testMap4.get("k1").isDefined) + assert(testMap4.get("k1").get === "v1") + } + } + + /** Test thread safety of a Scala mutable map. */ + def testMapThreadSafety(hashMapConstructor: => mutable.Map[String, String]) { + def newMap() = hashMapConstructor + val name = newMap().getClass.getSimpleName + val testMap = newMap() + @volatile var error = false + + def getRandomKey(m: mutable.Map[String, String]): Option[String] = { + val keys = testMap.keysIterator.toSeq + if (keys.nonEmpty) { + Some(keys(Random.nextInt(keys.size))) + } else { + None + } + } + + val threads = (1 to 25).map(i => new Thread() { + override def run() { + try { + for (j <- 1 to 1000) { + Random.nextInt(3) match { + case 0 => + testMap(Random.nextString(10)) = Random.nextDouble().toString // put + case 1 => + getRandomKey(testMap).map(testMap.get) // get + case 2 => + getRandomKey(testMap).map(testMap.remove) // remove + } + } + } catch { + case t: Throwable => + error = true + throw t + } + } + }) + + test(name + " - threading safety test") { + threads.map(_.start) + threads.map(_.join) + assert(!error) + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index d48b51aa69565..d043200f71a0b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -341,9 +341,11 @@ abstract class DStream[T: ClassTag] ( */ private[streaming] def clearMetadata(time: Time) { val oldRDDs = generatedRDDs.filter(_._1 <= (time - rememberDuration)) + logDebug("Clearing references to old RDDs: [" + + oldRDDs.map(x => s"${x._1} -> ${x._2.id}").mkString(", ") + "]") generatedRDDs --= oldRDDs.keys if (ssc.conf.getBoolean("spark.streaming.unpersist", false)) { - logDebug("Unpersisting old RDDs: " + oldRDDs.keys.mkString(", ")) + logDebug("Unpersisting old RDDs: " + oldRDDs.values.map(_.id).mkString(", ")) oldRDDs.values.foreach(_.unpersist(false)) } logDebug("Cleared " + oldRDDs.size + " RDDs that were older than " + From 83ac9a4bbf272028d0c4639cbd1e12022b9ae77a Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 8 Apr 2014 00:00:17 -0700 Subject: [PATCH 025/641] [SPARK-1331] Added graceful shutdown to Spark Streaming Current version of StreamingContext.stop() directly kills all the data receivers (NetworkReceiver) without waiting for the data already received to be persisted and processed. This PR provides the fix. Now, when the StreamingContext.stop() is called, the following sequence of steps will happen. 1. The driver will send a stop signal to all the active receivers. 2. Each receiver, when it gets a stop signal from the driver, first stop receiving more data, then waits for the thread that persists data blocks to BlockManager to finish persisting all receive data, and finally quits. 3. After all the receivers have stopped, the driver will wait for the Job Generator and Job Scheduler to finish processing all the received data. It also fixes the semantics of StreamingContext.start and stop. It will throw appropriate errors and warnings if stop() is called before start(), stop() is called twice, etc. Author: Tathagata Das Closes #247 from tdas/graceful-shutdown and squashes the following commits: 61c0016 [Tathagata Das] Updated MIMA binary check excludes. ae1d39b [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into graceful-shutdown 6b59cfc [Tathagata Das] Minor changes based on Andrew's comment on PR. d0b8d65 [Tathagata Das] Reduced time taken by graceful shutdown unit test. f55bc67 [Tathagata Das] Fix scalastyle c69b3a7 [Tathagata Das] Updates based on Patrick's comments. c43b8ae [Tathagata Das] Added graceful shutdown to Spark Streaming. --- project/MimaBuild.scala | 24 +-- .../apache/spark/streaming/Checkpoint.scala | 14 +- .../spark/streaming/StreamingContext.scala | 48 +++++- .../api/java/JavaStreamingContext.scala | 12 +- .../dstream/NetworkInputDStream.scala | 151 +++++++++++------ .../dstream/SocketInputDStream.scala | 1 - .../streaming/receivers/ActorReceiver.scala | 2 +- .../streaming/scheduler/JobGenerator.scala | 124 ++++++++++---- .../streaming/scheduler/JobScheduler.scala | 56 ++++--- .../scheduler/NetworkInputTracker.scala | 154 ++++++++++-------- .../apache/spark/streaming/util/Clock.scala | 5 +- .../spark/streaming/util/RecurringTimer.scala | 62 +++++-- .../streaming/BasicOperationsSuite.scala | 4 +- .../streaming/StreamingContextSuite.scala | 108 ++++++++++-- .../spark/streaming/TestSuiteBase.scala | 2 +- 15 files changed, 552 insertions(+), 215 deletions(-) diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index e7c9c47c960fa..5ea4817bfde18 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -58,17 +58,19 @@ object MimaBuild { SparkBuild.SPARK_VERSION match { case v if v.startsWith("1.0") => Seq( - excludePackage("org.apache.spark.api.java"), - excludePackage("org.apache.spark.streaming.api.java"), - excludePackage("org.apache.spark.mllib") - ) ++ - excludeSparkClass("rdd.ClassTags") ++ - excludeSparkClass("util.XORShiftRandom") ++ - excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ - excludeSparkClass("mllib.optimization.SquaredGradient") ++ - excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ - excludeSparkClass("mllib.regression.LassoWithSGD") ++ - excludeSparkClass("mllib.regression.LinearRegressionWithSGD") + excludePackage("org.apache.spark.api.java"), + excludePackage("org.apache.spark.streaming.api.java"), + excludePackage("org.apache.spark.mllib") + ) ++ + excludeSparkClass("rdd.ClassTags") ++ + excludeSparkClass("util.XORShiftRandom") ++ + excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ + excludeSparkClass("mllib.optimization.SquaredGradient") ++ + excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ + excludeSparkClass("mllib.regression.LassoWithSGD") ++ + excludeSparkClass("mllib.regression.LinearRegressionWithSGD") ++ + excludeSparkClass("streaming.dstream.NetworkReceiver") ++ + excludeSparkClass("streaming.dstream.NetworkReceiver#NetworkReceiverActor") case _ => Seq() } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index baf80fe2a91b7..93023e8dced57 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -194,19 +194,19 @@ class CheckpointWriter( } } - def stop() { - synchronized { - if (stopped) { - return - } - stopped = true - } + def stop(): Unit = synchronized { + if (stopped) return + executor.shutdown() val startTime = System.currentTimeMillis() val terminated = executor.awaitTermination(10, java.util.concurrent.TimeUnit.SECONDS) + if (!terminated) { + executor.shutdownNow() + } val endTime = System.currentTimeMillis() logInfo("CheckpointWriter executor terminated ? " + terminated + ", waited for " + (endTime - startTime) + " ms.") + stopped = true } private def fs = synchronized { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index e198c69470c1f..a4e236c65ff86 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -158,6 +158,15 @@ class StreamingContext private[streaming] ( private[streaming] val waiter = new ContextWaiter + /** Enumeration to identify current state of the StreamingContext */ + private[streaming] object StreamingContextState extends Enumeration { + type CheckpointState = Value + val Initialized, Started, Stopped = Value + } + + import StreamingContextState._ + private[streaming] var state = Initialized + /** * Return the associated Spark context */ @@ -405,9 +414,18 @@ class StreamingContext private[streaming] ( /** * Start the execution of the streams. */ - def start() = synchronized { + def start(): Unit = synchronized { + // Throw exception if the context has already been started once + // or if a stopped context is being started again + if (state == Started) { + throw new SparkException("StreamingContext has already been started") + } + if (state == Stopped) { + throw new SparkException("StreamingContext has already been stopped") + } validate() scheduler.start() + state = Started } /** @@ -428,14 +446,38 @@ class StreamingContext private[streaming] ( } /** - * Stop the execution of the streams. + * Stop the execution of the streams immediately (does not wait for all received data + * to be processed). * @param stopSparkContext Stop the associated SparkContext or not + * */ def stop(stopSparkContext: Boolean = true): Unit = synchronized { - scheduler.stop() + stop(stopSparkContext, false) + } + + /** + * Stop the execution of the streams, with option of ensuring all received data + * has been processed. + * @param stopSparkContext Stop the associated SparkContext or not + * @param stopGracefully Stop gracefully by waiting for the processing of all + * received data to be completed + */ + def stop(stopSparkContext: Boolean, stopGracefully: Boolean): Unit = synchronized { + // Warn (but not fail) if context is stopped twice, + // or context is stopped before starting + if (state == Initialized) { + logWarning("StreamingContext has not been started yet") + return + } + if (state == Stopped) { + logWarning("StreamingContext has already been stopped") + return + } // no need to throw an exception as its okay to stop twice + scheduler.stop(stopGracefully) logInfo("StreamingContext stopped successfully") waiter.notifyStop() if (stopSparkContext) sc.stop() + state = Stopped } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index b705d2ec9a58e..c800602d0959b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -509,8 +509,16 @@ class JavaStreamingContext(val ssc: StreamingContext) { * Stop the execution of the streams. * @param stopSparkContext Stop the associated SparkContext or not */ - def stop(stopSparkContext: Boolean): Unit = { - ssc.stop(stopSparkContext) + def stop(stopSparkContext: Boolean) = ssc.stop(stopSparkContext) + + /** + * Stop the execution of the streams. + * @param stopSparkContext Stop the associated SparkContext or not + * @param stopGracefully Stop gracefully by waiting for the processing of all + * received data to be completed + */ + def stop(stopSparkContext: Boolean, stopGracefully: Boolean) = { + ssc.stop(stopSparkContext, stopGracefully) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index 72ad0bae75bfb..d19a635fe8eca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import java.util.concurrent.ArrayBlockingQueue +import java.util.concurrent.{TimeUnit, ArrayBlockingQueue} import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer @@ -34,6 +34,7 @@ import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.rdd.{RDD, BlockRDD} import org.apache.spark.storage.{BlockId, StorageLevel, StreamBlockId} import org.apache.spark.streaming.scheduler.{DeregisterReceiver, AddBlocks, RegisterReceiver} +import org.apache.spark.util.AkkaUtils /** * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] @@ -69,7 +70,7 @@ abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingConte // then this returns an empty RDD. This may happen when recovering from a // master failure if (validTime >= graph.startTime) { - val blockIds = ssc.scheduler.networkInputTracker.getBlockIds(id, validTime) + val blockIds = ssc.scheduler.networkInputTracker.getBlocks(id, validTime) Some(new BlockRDD[T](ssc.sc, blockIds)) } else { Some(new BlockRDD[T](ssc.sc, Array[BlockId]())) @@ -79,7 +80,7 @@ abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingConte private[streaming] sealed trait NetworkReceiverMessage -private[streaming] case class StopReceiver(msg: String) extends NetworkReceiverMessage +private[streaming] case class StopReceiver() extends NetworkReceiverMessage private[streaming] case class ReportBlock(blockId: BlockId, metadata: Any) extends NetworkReceiverMessage private[streaming] case class ReportError(msg: String) extends NetworkReceiverMessage @@ -90,13 +91,31 @@ private[streaming] case class ReportError(msg: String) extends NetworkReceiverMe */ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging { + /** Local SparkEnv */ lazy protected val env = SparkEnv.get + /** Remote Akka actor for the NetworkInputTracker */ + lazy protected val trackerActor = { + val ip = env.conf.get("spark.driver.host", "localhost") + val port = env.conf.getInt("spark.driver.port", 7077) + val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) + env.actorSystem.actorSelection(url) + } + + /** Akka actor for receiving messages from the NetworkInputTracker in the driver */ lazy protected val actor = env.actorSystem.actorOf( Props(new NetworkReceiverActor()), "NetworkReceiver-" + streamId) + /** Timeout for Akka actor messages */ + lazy protected val askTimeout = AkkaUtils.askTimeout(env.conf) + + /** Thread that starts the receiver and stays blocked while data is being received */ lazy protected val receivingThread = Thread.currentThread() + /** Exceptions that occurs while receiving data */ + protected lazy val exceptions = new ArrayBuffer[Exception] + + /** Identifier of the stream this receiver is associated with */ protected var streamId: Int = -1 /** @@ -112,7 +131,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging def getLocationPreference() : Option[String] = None /** - * Starts the receiver. First is accesses all the lazy members to + * Start the receiver. First is accesses all the lazy members to * materialize them. Then it calls the user-defined onStart() method to start * other threads, etc required to receiver the data. */ @@ -124,83 +143,107 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging receivingThread // Call user-defined onStart() + logInfo("Starting receiver") onStart() + + // Wait until interrupt is called on this thread + while(true) Thread.sleep(100000) } catch { case ie: InterruptedException => - logInfo("Receiving thread interrupted") + logInfo("Receiving thread has been interrupted, receiver " + streamId + " stopped") case e: Exception => - stopOnError(e) + logError("Error receiving data in receiver " + streamId, e) + exceptions += e + } + + // Call user-defined onStop() + logInfo("Stopping receiver") + try { + onStop() + } catch { + case e: Exception => + logError("Error stopping receiver " + streamId, e) + exceptions += e + } + + val message = if (exceptions.isEmpty) { + null + } else if (exceptions.size == 1) { + val e = exceptions.head + "Exception in receiver " + streamId + ": " + e.getMessage + "\n" + e.getStackTraceString + } else { + "Multiple exceptions in receiver " + streamId + "(" + exceptions.size + "):\n" + exceptions.zipWithIndex.map { + case (e, i) => "Exception " + i + ": " + e.getMessage + "\n" + e.getStackTraceString + }.mkString("\n") } + logInfo("Deregistering receiver " + streamId) + val future = trackerActor.ask(DeregisterReceiver(streamId, message))(askTimeout) + Await.result(future, askTimeout) + logInfo("Deregistered receiver " + streamId) + env.actorSystem.stop(actor) + logInfo("Stopped receiver " + streamId) } /** - * Stops the receiver. First it interrupts the main receiving thread, - * that is, the thread that called receiver.start(). Then it calls the user-defined - * onStop() method to stop other threads and/or do cleanup. + * Stop the receiver. First it interrupts the main receiving thread, + * that is, the thread that called receiver.start(). */ def stop() { + // Stop receiving by interrupting the receiving thread receivingThread.interrupt() - onStop() - // TODO: terminate the actor + logInfo("Interrupted receiving thread " + receivingThread + " for stopping") } /** - * Stops the receiver and reports exception to the tracker. + * Stop the receiver and reports exception to the tracker. * This should be called whenever an exception is to be handled on any thread * of the receiver. */ protected def stopOnError(e: Exception) { logError("Error receiving data", e) + exceptions += e stop() - actor ! ReportError(e.toString) } - /** - * Pushes a block (as an ArrayBuffer filled with data) into the block manager. + * Push a block (as an ArrayBuffer filled with data) into the block manager. */ def pushBlock(blockId: BlockId, arrayBuffer: ArrayBuffer[T], metadata: Any, level: StorageLevel) { env.blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], level) - actor ! ReportBlock(blockId, metadata) + trackerActor ! AddBlocks(streamId, Array(blockId), metadata) + logDebug("Pushed block " + blockId) } /** - * Pushes a block (as bytes) into the block manager. + * Push a block (as bytes) into the block manager. */ def pushBlock(blockId: BlockId, bytes: ByteBuffer, metadata: Any, level: StorageLevel) { env.blockManager.putBytes(blockId, bytes, level) - actor ! ReportBlock(blockId, metadata) + trackerActor ! AddBlocks(streamId, Array(blockId), metadata) + } + + /** Set the ID of the DStream that this receiver is associated with */ + protected[streaming] def setStreamId(id: Int) { + streamId = id } /** A helper actor that communicates with the NetworkInputTracker */ private class NetworkReceiverActor extends Actor { - logInfo("Attempting to register with tracker") - val ip = env.conf.get("spark.driver.host", "localhost") - val port = env.conf.getInt("spark.driver.port", 7077) - val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) - val tracker = env.actorSystem.actorSelection(url) - val timeout = 5.seconds override def preStart() { - val future = tracker.ask(RegisterReceiver(streamId, self))(timeout) - Await.result(future, timeout) + logInfo("Registered receiver " + streamId) + val future = trackerActor.ask(RegisterReceiver(streamId, self))(askTimeout) + Await.result(future, askTimeout) } override def receive() = { - case ReportBlock(blockId, metadata) => - tracker ! AddBlocks(streamId, Array(blockId), metadata) - case ReportError(msg) => - tracker ! DeregisterReceiver(streamId, msg) - case StopReceiver(msg) => + case StopReceiver => + logInfo("Received stop signal") stop() - tracker ! DeregisterReceiver(streamId, msg) } } - protected[streaming] def setStreamId(id: Int) { - streamId = id - } - /** * Batches objects created by a [[org.apache.spark.streaming.dstream.NetworkReceiver]] and puts * them into appropriately named blocks at regular intervals. This class starts two threads, @@ -214,23 +257,26 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging val clock = new SystemClock() val blockInterval = env.conf.getLong("spark.streaming.blockInterval", 200) - val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer) + val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer, + "BlockGenerator") val blockStorageLevel = storageLevel val blocksForPushing = new ArrayBlockingQueue[Block](1000) val blockPushingThread = new Thread() { override def run() { keepPushingBlocks() } } var currentBuffer = new ArrayBuffer[T] + var stopped = false def start() { blockIntervalTimer.start() blockPushingThread.start() - logInfo("Data handler started") + logInfo("Started BlockGenerator") } def stop() { - blockIntervalTimer.stop() - blockPushingThread.interrupt() - logInfo("Data handler stopped") + blockIntervalTimer.stop(false) + stopped = true + blockPushingThread.join() + logInfo("Stopped BlockGenerator") } def += (obj: T): Unit = synchronized { @@ -248,24 +294,35 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging } } catch { case ie: InterruptedException => - logInfo("Block interval timer thread interrupted") + logInfo("Block updating timer thread was interrupted") case e: Exception => - NetworkReceiver.this.stop() + NetworkReceiver.this.stopOnError(e) } } private def keepPushingBlocks() { - logInfo("Block pushing thread started") + logInfo("Started block pushing thread") try { - while(true) { + while(!stopped) { + Option(blocksForPushing.poll(100, TimeUnit.MILLISECONDS)) match { + case Some(block) => + NetworkReceiver.this.pushBlock(block.id, block.buffer, block.metadata, storageLevel) + case None => + } + } + // Push out the blocks that are still left + logInfo("Pushing out the last " + blocksForPushing.size() + " blocks") + while (!blocksForPushing.isEmpty) { val block = blocksForPushing.take() NetworkReceiver.this.pushBlock(block.id, block.buffer, block.metadata, storageLevel) + logInfo("Blocks left to push " + blocksForPushing.size()) } + logInfo("Stopped blocks pushing thread") } catch { case ie: InterruptedException => - logInfo("Block pushing thread interrupted") + logInfo("Block pushing thread was interrupted") case e: Exception => - NetworkReceiver.this.stop() + NetworkReceiver.this.stopOnError(e) } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala index 2cdd13f205313..63d94d1cc670a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala @@ -67,7 +67,6 @@ class SocketReceiver[T: ClassTag]( protected def onStop() { blockGenerator.stop() } - } private[streaming] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala index bd78bae8a5c51..44eb2750c6c7a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala @@ -174,10 +174,10 @@ private[streaming] class ActorReceiver[T: ClassTag]( blocksGenerator.start() supervisor logInfo("Supervision tree for receivers initialized at:" + supervisor.path) + } protected def onStop() = { supervisor ! PoisonPill } - } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index c7306248b1950..92d885c4bc5a5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -39,16 +39,22 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { private val ssc = jobScheduler.ssc private val graph = ssc.graph + val clock = { val clockClass = ssc.sc.conf.get( "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") Class.forName(clockClass).newInstance().asInstanceOf[Clock] } + private val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds, - longTime => eventActor ! GenerateJobs(new Time(longTime))) - private lazy val checkpointWriter = - if (ssc.checkpointDuration != null && ssc.checkpointDir != null) { - new CheckpointWriter(this, ssc.conf, ssc.checkpointDir, ssc.sparkContext.hadoopConfiguration) + longTime => eventActor ! GenerateJobs(new Time(longTime)), "JobGenerator") + + // This is marked lazy so that this is initialized after checkpoint duration has been set + // in the context and the generator has been started. + private lazy val shouldCheckpoint = ssc.checkpointDuration != null && ssc.checkpointDir != null + + private lazy val checkpointWriter = if (shouldCheckpoint) { + new CheckpointWriter(this, ssc.conf, ssc.checkpointDir, ssc.sparkContext.hadoopConfiguration) } else { null } @@ -57,17 +63,16 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { // This not being null means the scheduler has been started and not stopped private var eventActor: ActorRef = null + // last batch whose completion,checkpointing and metadata cleanup has been completed + private var lastProcessedBatch: Time = null + /** Start generation of jobs */ - def start() = synchronized { - if (eventActor != null) { - throw new SparkException("JobGenerator already started") - } + def start(): Unit = synchronized { + if (eventActor != null) return // generator has already been started eventActor = ssc.env.actorSystem.actorOf(Props(new Actor { def receive = { - case event: JobGeneratorEvent => - logDebug("Got event of type " + event.getClass.getName) - processEvent(event) + case event: JobGeneratorEvent => processEvent(event) } }), "JobGenerator") if (ssc.isCheckpointPresent) { @@ -77,30 +82,79 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { } } - /** Stop generation of jobs */ - def stop() = synchronized { - if (eventActor != null) { - timer.stop() - ssc.env.actorSystem.stop(eventActor) - if (checkpointWriter != null) checkpointWriter.stop() - ssc.graph.stop() - logInfo("JobGenerator stopped") + /** + * Stop generation of jobs. processReceivedData = true makes this wait until jobs + * of current ongoing time interval has been generated, processed and corresponding + * checkpoints written. + */ + def stop(processReceivedData: Boolean): Unit = synchronized { + if (eventActor == null) return // generator has already been stopped + + if (processReceivedData) { + logInfo("Stopping JobGenerator gracefully") + val timeWhenStopStarted = System.currentTimeMillis() + val stopTimeout = 10 * ssc.graph.batchDuration.milliseconds + val pollTime = 100 + + // To prevent graceful stop to get stuck permanently + def hasTimedOut = { + val timedOut = System.currentTimeMillis() - timeWhenStopStarted > stopTimeout + if (timedOut) logWarning("Timed out while stopping the job generator") + timedOut + } + + // Wait until all the received blocks in the network input tracker has + // been consumed by network input DStreams, and jobs have been generated with them + logInfo("Waiting for all received blocks to be consumed for job generation") + while(!hasTimedOut && jobScheduler.networkInputTracker.hasMoreReceivedBlockIds) { + Thread.sleep(pollTime) + } + logInfo("Waited for all received blocks to be consumed for job generation") + + // Stop generating jobs + val stopTime = timer.stop(false) + graph.stop() + logInfo("Stopped generation timer") + + // Wait for the jobs to complete and checkpoints to be written + def haveAllBatchesBeenProcessed = { + lastProcessedBatch != null && lastProcessedBatch.milliseconds == stopTime + } + logInfo("Waiting for jobs to be processed and checkpoints to be written") + while (!hasTimedOut && !haveAllBatchesBeenProcessed) { + Thread.sleep(pollTime) + } + logInfo("Waited for jobs to be processed and checkpoints to be written") + } else { + logInfo("Stopping JobGenerator immediately") + // Stop timer and graph immediately, ignore unprocessed data and pending jobs + timer.stop(true) + graph.stop() } + + // Stop the actor and checkpoint writer + if (shouldCheckpoint) checkpointWriter.stop() + ssc.env.actorSystem.stop(eventActor) + logInfo("Stopped JobGenerator") } /** - * On batch completion, clear old metadata and checkpoint computation. + * Callback called when a batch has been completely processed. */ def onBatchCompletion(time: Time) { eventActor ! ClearMetadata(time) } - + + /** + * Callback called when the checkpoint of a batch has been written. + */ def onCheckpointCompletion(time: Time) { eventActor ! ClearCheckpointData(time) } /** Processes all events */ private def processEvent(event: JobGeneratorEvent) { + logDebug("Got event " + event) event match { case GenerateJobs(time) => generateJobs(time) case ClearMetadata(time) => clearMetadata(time) @@ -114,7 +168,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { val startTime = new Time(timer.getStartTime()) graph.start(startTime - graph.batchDuration) timer.start(startTime.milliseconds) - logInfo("JobGenerator started at " + startTime) + logInfo("Started JobGenerator at " + startTime) } /** Restarts the generator based on the information in checkpoint */ @@ -152,15 +206,17 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { // Restart the timer timer.start(restartTime.milliseconds) - logInfo("JobGenerator restarted at " + restartTime) + logInfo("Restarted JobGenerator at " + restartTime) } /** Generate jobs and perform checkpoint for the given `time`. */ private def generateJobs(time: Time) { SparkEnv.set(ssc.env) Try(graph.generateJobs(time)) match { - case Success(jobs) => jobScheduler.runJobs(time, jobs) - case Failure(e) => jobScheduler.reportError("Error generating jobs for time " + time, e) + case Success(jobs) => + jobScheduler.runJobs(time, jobs) + case Failure(e) => + jobScheduler.reportError("Error generating jobs for time " + time, e) } eventActor ! DoCheckpoint(time) } @@ -168,20 +224,32 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { /** Clear DStream metadata for the given `time`. */ private def clearMetadata(time: Time) { ssc.graph.clearMetadata(time) - eventActor ! DoCheckpoint(time) + + // If checkpointing is enabled, then checkpoint, + // else mark batch to be fully processed + if (shouldCheckpoint) { + eventActor ! DoCheckpoint(time) + } else { + markBatchFullyProcessed(time) + } } /** Clear DStream checkpoint data for the given `time`. */ private def clearCheckpointData(time: Time) { ssc.graph.clearCheckpointData(time) + markBatchFullyProcessed(time) } /** Perform checkpoint for the give `time`. */ - private def doCheckpoint(time: Time) = synchronized { - if (checkpointWriter != null && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) { + private def doCheckpoint(time: Time) { + if (shouldCheckpoint && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) { logInfo("Checkpointing graph for time " + time) ssc.graph.updateCheckpointData(time) checkpointWriter.write(new Checkpoint(ssc, time)) } } + + private def markBatchFullyProcessed(time: Time) { + lastProcessedBatch = time + } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index de675d3c7fb94..04e0a6a283cfb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -39,7 +39,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { private val jobSets = new ConcurrentHashMap[Time, JobSet] private val numConcurrentJobs = ssc.conf.getInt("spark.streaming.concurrentJobs", 1) - private val executor = Executors.newFixedThreadPool(numConcurrentJobs) + private val jobExecutor = Executors.newFixedThreadPool(numConcurrentJobs) private val jobGenerator = new JobGenerator(this) val clock = jobGenerator.clock val listenerBus = new StreamingListenerBus() @@ -50,36 +50,54 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { private var eventActor: ActorRef = null - def start() = synchronized { - if (eventActor != null) { - throw new SparkException("JobScheduler already started") - } + def start(): Unit = synchronized { + if (eventActor != null) return // scheduler has already been started + logDebug("Starting JobScheduler") eventActor = ssc.env.actorSystem.actorOf(Props(new Actor { def receive = { case event: JobSchedulerEvent => processEvent(event) } }), "JobScheduler") + listenerBus.start() networkInputTracker = new NetworkInputTracker(ssc) networkInputTracker.start() - Thread.sleep(1000) jobGenerator.start() - logInfo("JobScheduler started") + logInfo("Started JobScheduler") } - def stop() = synchronized { - if (eventActor != null) { - jobGenerator.stop() - networkInputTracker.stop() - executor.shutdown() - if (!executor.awaitTermination(2, TimeUnit.SECONDS)) { - executor.shutdownNow() - } - listenerBus.stop() - ssc.env.actorSystem.stop(eventActor) - logInfo("JobScheduler stopped") + def stop(processAllReceivedData: Boolean): Unit = synchronized { + if (eventActor == null) return // scheduler has already been stopped + logDebug("Stopping JobScheduler") + + // First, stop receiving + networkInputTracker.stop() + + // Second, stop generating jobs. If it has to process all received data, + // then this will wait for all the processing through JobScheduler to be over. + jobGenerator.stop(processAllReceivedData) + + // Stop the executor for receiving new jobs + logDebug("Stopping job executor") + jobExecutor.shutdown() + + // Wait for the queued jobs to complete if indicated + val terminated = if (processAllReceivedData) { + jobExecutor.awaitTermination(1, TimeUnit.HOURS) // just a very large period of time + } else { + jobExecutor.awaitTermination(2, TimeUnit.SECONDS) } + if (!terminated) { + jobExecutor.shutdownNow() + } + logDebug("Stopped job executor") + + // Stop everything else + listenerBus.stop() + ssc.env.actorSystem.stop(eventActor) + eventActor = null + logInfo("Stopped JobScheduler") } def runJobs(time: Time, jobs: Seq[Job]) { @@ -88,7 +106,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { } else { val jobSet = new JobSet(time, jobs) jobSets.put(time, jobSet) - jobSet.jobs.foreach(job => executor.execute(new JobHandler(job))) + jobSet.jobs.foreach(job => jobExecutor.execute(new JobHandler(job))) logInfo("Added jobs for time " + time) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala index cad68e248ab29..067e804202236 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala @@ -17,20 +17,14 @@ package org.apache.spark.streaming.scheduler -import org.apache.spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver} -import org.apache.spark.streaming.dstream.{StopReceiver, ReportBlock, ReportError} -import org.apache.spark.{SparkException, Logging, SparkEnv} -import org.apache.spark.SparkContext._ - -import scala.collection.mutable.HashMap -import scala.collection.mutable.Queue -import scala.concurrent.duration._ +import scala.collection.mutable.{HashMap, Queue, SynchronizedMap} import akka.actor._ -import akka.pattern.ask -import akka.dispatch._ +import org.apache.spark.{Logging, SparkEnv, SparkException} +import org.apache.spark.SparkContext._ import org.apache.spark.storage.BlockId -import org.apache.spark.streaming.{Time, StreamingContext} +import org.apache.spark.streaming.{StreamingContext, Time} +import org.apache.spark.streaming.dstream.{NetworkReceiver, StopReceiver} import org.apache.spark.util.AkkaUtils private[streaming] sealed trait NetworkInputTrackerMessage @@ -52,8 +46,8 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { val networkInputStreams = ssc.graph.getNetworkInputStreams() val networkInputStreamMap = Map(networkInputStreams.map(x => (x.id, x)): _*) val receiverExecutor = new ReceiverExecutor() - val receiverInfo = new HashMap[Int, ActorRef] - val receivedBlockIds = new HashMap[Int, Queue[BlockId]] + val receiverInfo = new HashMap[Int, ActorRef] with SynchronizedMap[Int, ActorRef] + val receivedBlockIds = new HashMap[Int, Queue[BlockId]] with SynchronizedMap[Int, Queue[BlockId]] val timeout = AkkaUtils.askTimeout(ssc.conf) @@ -63,7 +57,7 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { var currentTime: Time = null /** Start the actor and receiver execution thread. */ - def start() { + def start() = synchronized { if (actor != null) { throw new SparkException("NetworkInputTracker already started") } @@ -77,72 +71,99 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { } /** Stop the receiver execution thread. */ - def stop() { + def stop() = synchronized { if (!networkInputStreams.isEmpty && actor != null) { - receiverExecutor.interrupt() - receiverExecutor.stopReceivers() + // First, stop the receivers + receiverExecutor.stop() + + // Finally, stop the actor ssc.env.actorSystem.stop(actor) + actor = null logInfo("NetworkInputTracker stopped") } } - /** Return all the blocks received from a receiver. */ - def getBlockIds(receiverId: Int, time: Time): Array[BlockId] = synchronized { - val queue = receivedBlockIds.synchronized { - receivedBlockIds.getOrElse(receiverId, new Queue[BlockId]()) + /** Register a receiver */ + def registerReceiver(streamId: Int, receiverActor: ActorRef, sender: ActorRef) { + if (!networkInputStreamMap.contains(streamId)) { + throw new Exception("Register received for unexpected id " + streamId) } - val result = queue.synchronized { - queue.dequeueAll(x => true) - } - logInfo("Stream " + receiverId + " received " + result.size + " blocks") - result.toArray + receiverInfo += ((streamId, receiverActor)) + logInfo("Registered receiver for network stream " + streamId + " from " + sender.path.address) + } + + /** Deregister a receiver */ + def deregisterReceiver(streamId: Int, message: String) { + receiverInfo -= streamId + logError("Deregistered receiver for network stream " + streamId + " with message:\n" + message) + } + + /** Get all the received blocks for the given stream. */ + def getBlocks(streamId: Int, time: Time): Array[BlockId] = { + val queue = receivedBlockIds.getOrElseUpdate(streamId, new Queue[BlockId]()) + val result = queue.dequeueAll(x => true).toArray + logInfo("Stream " + streamId + " received " + result.size + " blocks") + result + } + + /** Add new blocks for the given stream */ + def addBlocks(streamId: Int, blockIds: Seq[BlockId], metadata: Any) = { + val queue = receivedBlockIds.getOrElseUpdate(streamId, new Queue[BlockId]) + queue ++= blockIds + networkInputStreamMap(streamId).addMetadata(metadata) + logDebug("Stream " + streamId + " received new blocks: " + blockIds.mkString("[", ", ", "]")) + } + + /** Check if any blocks are left to be processed */ + def hasMoreReceivedBlockIds: Boolean = { + !receivedBlockIds.forall(_._2.isEmpty) } /** Actor to receive messages from the receivers. */ private class NetworkInputTrackerActor extends Actor { def receive = { - case RegisterReceiver(streamId, receiverActor) => { - if (!networkInputStreamMap.contains(streamId)) { - throw new Exception("Register received for unexpected id " + streamId) - } - receiverInfo += ((streamId, receiverActor)) - logInfo("Registered receiver for network stream " + streamId + " from " - + sender.path.address) + case RegisterReceiver(streamId, receiverActor) => + registerReceiver(streamId, receiverActor, sender) + sender ! true + case AddBlocks(streamId, blockIds, metadata) => + addBlocks(streamId, blockIds, metadata) + case DeregisterReceiver(streamId, message) => + deregisterReceiver(streamId, message) sender ! true - } - case AddBlocks(streamId, blockIds, metadata) => { - val tmp = receivedBlockIds.synchronized { - if (!receivedBlockIds.contains(streamId)) { - receivedBlockIds += ((streamId, new Queue[BlockId])) - } - receivedBlockIds(streamId) - } - tmp.synchronized { - tmp ++= blockIds - } - networkInputStreamMap(streamId).addMetadata(metadata) - } - case DeregisterReceiver(streamId, msg) => { - receiverInfo -= streamId - logError("De-registered receiver for network stream " + streamId - + " with message " + msg) - // TODO: Do something about the corresponding NetworkInputDStream - } } } /** This thread class runs all the receivers on the cluster. */ - class ReceiverExecutor extends Thread { - val env = ssc.env - - override def run() { - try { - SparkEnv.set(env) - startReceivers() - } catch { - case ie: InterruptedException => logInfo("ReceiverExecutor interrupted") - } finally { - stopReceivers() + class ReceiverExecutor { + @transient val env = ssc.env + @transient val thread = new Thread() { + override def run() { + try { + SparkEnv.set(env) + startReceivers() + } catch { + case ie: InterruptedException => logInfo("ReceiverExecutor interrupted") + } + } + } + + def start() { + thread.start() + } + + def stop() { + // Send the stop signal to all the receivers + stopReceivers() + + // Wait for the Spark job that runs the receivers to be over + // That is, for the receivers to quit gracefully. + thread.join(10000) + + // Check if all the receivers have been deregistered or not + if (!receiverInfo.isEmpty) { + logWarning("All of the receivers have not deregistered, " + receiverInfo) + } else { + logInfo("All of the receivers have deregistered successfully") } } @@ -150,7 +171,7 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { * Get the receivers from the NetworkInputDStreams, distributes them to the * worker nodes as a parallel collection, and runs them. */ - def startReceivers() { + private def startReceivers() { val receivers = networkInputStreams.map(nis => { val rcvr = nis.getReceiver() rcvr.setStreamId(nis.id) @@ -186,13 +207,16 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { } // Distribute the receivers and start them + logInfo("Starting " + receivers.length + " receivers") ssc.sparkContext.runJob(tempRDD, startReceiver) + logInfo("All of the receivers have been terminated") } /** Stops the receivers. */ - def stopReceivers() { + private def stopReceivers() { // Signal the receivers to stop receiverInfo.values.foreach(_ ! StopReceiver) + logInfo("Sent stop signal to all " + receiverInfo.size + " receivers") } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala index c3a849d2769a7..c5ef2cc8c390d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala @@ -48,14 +48,11 @@ class SystemClock() extends Clock { minPollTime } } - - + while (true) { currentTime = System.currentTimeMillis() waitTime = targetTime - currentTime - if (waitTime <= 0) { - return currentTime } val sleepTime = diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala index 559c2473851b3..f71938ac55ccb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala @@ -17,44 +17,84 @@ package org.apache.spark.streaming.util +import org.apache.spark.Logging + private[streaming] -class RecurringTimer(val clock: Clock, val period: Long, val callback: (Long) => Unit) { +class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name: String) + extends Logging { - private val thread = new Thread("RecurringTimer") { + private val thread = new Thread("RecurringTimer - " + name) { + setDaemon(true) override def run() { loop } } - - private var nextTime = 0L + @volatile private var prevTime = -1L + @volatile private var nextTime = -1L + @volatile private var stopped = false + + /** + * Get the time when this timer will fire if it is started right now. + * The time will be a multiple of this timer's period and more than + * current system time. + */ def getStartTime(): Long = { (math.floor(clock.currentTime.toDouble / period) + 1).toLong * period } + /** + * Get the time when the timer will fire if it is restarted right now. + * This time depends on when the timer was started the first time, and was stopped + * for whatever reason. The time must be a multiple of this timer's period and + * more than current time. + */ def getRestartTime(originalStartTime: Long): Long = { val gap = clock.currentTime - originalStartTime (math.floor(gap.toDouble / period).toLong + 1) * period + originalStartTime } - def start(startTime: Long): Long = { + /** + * Start at the given start time. + */ + def start(startTime: Long): Long = synchronized { nextTime = startTime thread.start() + logInfo("Started timer for " + name + " at time " + nextTime) nextTime } + /** + * Start at the earliest time it can start based on the period. + */ def start(): Long = { start(getStartTime()) } - def stop() { - thread.interrupt() + /** + * Stop the timer, and return the last time the callback was made. + * interruptTimer = true will interrupt the callback + * if it is in progress (not guaranteed to give correct time in this case). + */ + def stop(interruptTimer: Boolean): Long = synchronized { + if (!stopped) { + stopped = true + if (interruptTimer) thread.interrupt() + thread.join() + logInfo("Stopped timer for " + name + " after time " + prevTime) + } + prevTime } - + + /** + * Repeatedly call the callback every interval. + */ private def loop() { try { - while (true) { + while (!stopped) { clock.waitTillTime(nextTime) callback(nextTime) + prevTime = nextTime nextTime += period + logDebug("Callback for " + name + " called at time " + prevTime) } } catch { case e: InterruptedException => @@ -74,10 +114,10 @@ object RecurringTimer { println("" + currentTime + ": " + (currentTime - lastRecurTime)) lastRecurTime = currentTime } - val timer = new RecurringTimer(new SystemClock(), period, onRecur) + val timer = new RecurringTimer(new SystemClock(), period, onRecur, "Test") timer.start() Thread.sleep(30 * 1000) - timer.stop() + timer.stop(true) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index bcb0c28bf07a0..bb73dbf29b649 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -324,7 +324,7 @@ class BasicOperationsSuite extends TestSuiteBase { val updateStateOperation = (s: DStream[String]) => { val updateFunc = (values: Seq[Int], state: Option[Int]) => { - Some(values.foldLeft(0)(_ + _) + state.getOrElse(0)) + Some(values.sum + state.getOrElse(0)) } s.map(x => (x, 1)).updateStateByKey[Int](updateFunc) } @@ -359,7 +359,7 @@ class BasicOperationsSuite extends TestSuiteBase { // updateFunc clears a state when a StateObject is seen without new values twice in a row val updateFunc = (values: Seq[Int], state: Option[StateObject]) => { val stateObj = state.getOrElse(new StateObject) - values.foldLeft(0)(_ + _) match { + values.sum match { case 0 => stateObj.expireCounter += 1 // no new values case n => { // has new values, increment and reset expireCounter stateObj.counter += n diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 717da8e00462b..9cc27ef7f03b5 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -17,19 +17,22 @@ package org.apache.spark.streaming -import org.scalatest.{FunSuite, BeforeAndAfter} -import org.scalatest.exceptions.TestFailedDueToTimeoutException +import java.util.concurrent.atomic.AtomicInteger + +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException} +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.dstream.{DStream, NetworkReceiver} +import org.apache.spark.util.{MetadataCleaner, Utils} +import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.concurrent.Timeouts +import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.SpanSugar._ -import org.apache.spark.{SparkException, SparkConf, SparkContext} -import org.apache.spark.util.{Utils, MetadataCleaner} -import org.apache.spark.streaming.dstream.DStream -class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts { +class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts with Logging { val master = "local[2]" val appName = this.getClass.getSimpleName - val batchDuration = Seconds(1) + val batchDuration = Milliseconds(500) val sparkHome = "someDir" val envPair = "key" -> "value" val ttl = StreamingContext.DEFAULT_CLEANER_TTL + 100 @@ -108,19 +111,31 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) myConf.set("spark.cleaner.ttl", ttl.toString) val ssc1 = new StreamingContext(myConf, batchDuration) + addInputStream(ssc1).register + ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) assert(MetadataCleaner.getDelaySeconds(cp.sparkConf) === ttl) ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) assert(MetadataCleaner.getDelaySeconds(newCp.sparkConf) === ttl) - ssc = new StreamingContext(null, cp, null) + ssc = new StreamingContext(null, newCp, null) assert(MetadataCleaner.getDelaySeconds(ssc.conf) === ttl) } - test("start multiple times") { + test("start and stop state check") { ssc = new StreamingContext(master, appName, batchDuration) addInputStream(ssc).register + assert(ssc.state === ssc.StreamingContextState.Initialized) + ssc.start() + assert(ssc.state === ssc.StreamingContextState.Started) + ssc.stop() + assert(ssc.state === ssc.StreamingContextState.Stopped) + } + + test("start multiple times") { + ssc = new StreamingContext(master, appName, batchDuration) + addInputStream(ssc).register ssc.start() intercept[SparkException] { ssc.start() @@ -133,18 +148,61 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts { ssc.start() ssc.stop() ssc.stop() - ssc = null } + test("stop before start and start after stop") { + ssc = new StreamingContext(master, appName, batchDuration) + addInputStream(ssc).register + ssc.stop() // stop before start should not throw exception + ssc.start() + ssc.stop() + intercept[SparkException] { + ssc.start() // start after stop should throw exception + } + } + + test("stop only streaming context") { ssc = new StreamingContext(master, appName, batchDuration) sc = ssc.sparkContext addInputStream(ssc).register ssc.start() ssc.stop(false) - ssc = null assert(sc.makeRDD(1 to 100).collect().size === 100) ssc = new StreamingContext(sc, batchDuration) + addInputStream(ssc).register + ssc.start() + ssc.stop() + } + + test("stop gracefully") { + val conf = new SparkConf().setMaster(master).setAppName(appName) + conf.set("spark.cleaner.ttl", "3600") + sc = new SparkContext(conf) + for (i <- 1 to 4) { + logInfo("==================================") + ssc = new StreamingContext(sc, batchDuration) + var runningCount = 0 + TestReceiver.counter.set(1) + val input = ssc.networkStream(new TestReceiver) + input.count.foreachRDD(rdd => { + val count = rdd.first() + logInfo("Count = " + count) + runningCount += count.toInt + }) + ssc.start() + ssc.awaitTermination(500) + ssc.stop(stopSparkContext = false, stopGracefully = true) + logInfo("Running count = " + runningCount) + logInfo("TestReceiver.counter = " + TestReceiver.counter.get()) + assert(runningCount > 0) + assert( + (TestReceiver.counter.get() == runningCount + 1) || + (TestReceiver.counter.get() == runningCount + 2), + "Received records = " + TestReceiver.counter.get() + ", " + + "processed records = " + runningCount + ) + } } test("awaitTermination") { @@ -199,7 +257,6 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts { test("awaitTermination with error in job generation") { ssc = new StreamingContext(master, appName, batchDuration) val inputStream = addInputStream(ssc) - inputStream.transform(rdd => { throw new TestException("error in transform"); rdd }).register val exception = intercept[TestException] { ssc.start() @@ -215,4 +272,29 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts { } } -class TestException(msg: String) extends Exception(msg) \ No newline at end of file +class TestException(msg: String) extends Exception(msg) + +/** Custom receiver for testing whether all data received by a receiver gets processed or not */ +class TestReceiver extends NetworkReceiver[Int] { + protected lazy val blockGenerator = new BlockGenerator(StorageLevel.MEMORY_ONLY) + protected def onStart() { + blockGenerator.start() + logInfo("BlockGenerator started on thread " + receivingThread) + try { + while(true) { + blockGenerator += TestReceiver.counter.getAndIncrement + Thread.sleep(0) + } + } finally { + logInfo("Receiving stopped at count value of " + TestReceiver.counter.get()) + } + } + + protected def onStop() { + blockGenerator.stop() + } +} + +object TestReceiver { + val counter = new AtomicInteger(1) +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 201630672ab4c..aa2d5c2fc2454 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -277,7 +277,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms") assert(output.size === numExpectedOutput, "Unexpected number of outputs generated") - Thread.sleep(500) // Give some time for the forgetting old RDDs to complete + Thread.sleep(100) // Give some time for the forgetting old RDDs to complete } catch { case e: Exception => {e.printStackTrace(); throw e} } finally { From 6dc5f5849c0e0378abc6648c919412827d831641 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 8 Apr 2014 01:03:33 -0700 Subject: [PATCH 026/641] [SPARK-1396] Properly cleanup DAGScheduler on job cancellation. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, when jobs were cancelled, not all of the state in the DAGScheduler was cleaned up, leading to a slow memory leak in the DAGScheduler. As we expose easier ways to cancel jobs, it's more important to fix these issues. This commit also fixes a second and less serious problem, which is that previously, when a stage failed, not all of the appropriate stages were cancelled. See the "failure of stage used by two jobs" test for an example of this. This just meant that extra work was done, and is not a correctness problem. This commit adds 3 tests. “run shuffle with map stage failure” is a new test to more thoroughly test this functionality, and passes on both the old and new versions of the code. “trivial job cancellation” fails on the old code because all state wasn’t cleaned up correctly when jobs were cancelled (we didn’t remove the job from resultStageToJob). “failure of stage used by two jobs” fails on the old code because taskScheduler.cancelTasks wasn’t called for one of the stages (see test comments). This should be checked in before #246, which makes it easier to cancel stages / jobs. Author: Kay Ousterhout Closes #305 from kayousterhout/incremental_abort_fix and squashes the following commits: f33d844 [Kay Ousterhout] Mark review comments 9217080 [Kay Ousterhout] Properly cleanup DAGScheduler on job cancellation. --- .../apache/spark/scheduler/DAGScheduler.scala | 44 +++++---- .../spark/scheduler/DAGSchedulerSuite.scala | 92 ++++++++++++++++++- 2 files changed, 115 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 6368665f249ee..c96d7435a7ed4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -982,15 +982,7 @@ class DAGScheduler( if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { - val independentStages = removeJobAndIndependentStages(jobId) - independentStages.foreach(taskScheduler.cancelTasks) - val error = new SparkException("Job %d cancelled".format(jobId)) - val job = jobIdToActiveJob(jobId) - job.listener.jobFailed(error) - jobIdToStageIds -= jobId - activeJobs -= job - jobIdToActiveJob -= jobId - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) + failJobAndIndependentStages(jobIdToActiveJob(jobId), s"Job $jobId cancelled") } } @@ -1007,19 +999,39 @@ class DAGScheduler( stageToInfos(failedStage).completionTime = Some(System.currentTimeMillis()) for (resultStage <- dependentStages) { val job = resultStageToJob(resultStage) - val error = new SparkException("Job aborted: " + reason) - job.listener.jobFailed(error) - jobIdToStageIdsRemove(job.jobId) - jobIdToActiveJob -= resultStage.jobId - activeJobs -= job - resultStageToJob -= resultStage - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, failedStage.id))) + failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason") } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") } } + /** + * Fails a job and all stages that are only used by that job, and cleans up relevant state. + */ + private def failJobAndIndependentStages(job: ActiveJob, failureReason: String) { + val error = new SparkException(failureReason) + job.listener.jobFailed(error) + + // Cancel all tasks in independent stages. + val independentStages = removeJobAndIndependentStages(job.jobId) + independentStages.foreach(taskScheduler.cancelTasks) + + // Clean up remaining state we store for the job. + jobIdToActiveJob -= job.jobId + activeJobs -= job + jobIdToStageIds -= job.jobId + val resultStagesForJob = resultStageToJob.keySet.filter( + stage => resultStageToJob(stage).jobId == job.jobId) + if (resultStagesForJob.size != 1) { + logWarning( + s"${resultStagesForJob.size} result stages for job ${job.jobId} (expect exactly 1)") + } + resultStageToJob --= resultStagesForJob + + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) + } + /** * Return true if one of stage's ancestors is target. */ diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index ce567b0cde85d..2e3026bffba2f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler import scala.Tuple2 -import scala.collection.mutable.{HashMap, Map} +import scala.collection.mutable.{HashSet, HashMap, Map} import org.scalatest.{BeforeAndAfter, FunSuite} @@ -43,6 +43,10 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() + + /** Stages for which the DAGScheduler has called TaskScheduler.cancelTasks(). */ + val cancelledStages = new HashSet[Int]() + val taskScheduler = new TaskScheduler() { override def rootPool: Pool = null override def schedulingMode: SchedulingMode = SchedulingMode.NONE @@ -53,7 +57,9 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) taskSets += taskSet } - override def cancelTasks(stageId: Int) {} + override def cancelTasks(stageId: Int) { + cancelledStages += stageId + } override def setDAGScheduler(dagScheduler: DAGScheduler) = {} override def defaultParallelism() = 2 } @@ -91,6 +97,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont before { sc = new SparkContext("local", "DAGSchedulerSuite") taskSets.clear() + cancelledStages.clear() cacheLocations.clear() results.clear() mapOutputTracker = new MapOutputTrackerMaster(conf) @@ -174,15 +181,16 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont } } - /** Sends the rdd to the scheduler for scheduling. */ + /** Sends the rdd to the scheduler for scheduling and returns the job id. */ private def submit( rdd: RDD[_], partitions: Array[Int], func: (TaskContext, Iterator[_]) => _ = jobComputeFunc, allowLocal: Boolean = false, - listener: JobListener = listener) { + listener: JobListener = listener): Int = { val jobId = scheduler.nextJobId.getAndIncrement() runEvent(JobSubmitted(jobId, rdd, func, partitions, allowLocal, null, listener)) + return jobId } /** Sends TaskSetFailed to the scheduler. */ @@ -190,6 +198,11 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont runEvent(TaskSetFailed(taskSet, message)) } + /** Sends JobCancelled to the DAG scheduler. */ + private def cancel(jobId: Int) { + runEvent(JobCancelled(jobId)) + } + test("zero split job") { val rdd = makeRdd(0, Nil) var numResults = 0 @@ -248,7 +261,15 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont test("trivial job failure") { submit(makeRdd(1, Nil), Array(0)) failed(taskSets(0), "some failure") - assert(failure.getMessage === "Job aborted: some failure") + assert(failure.getMessage === "Job aborted due to stage failure: some failure") + assertDataStructuresEmpty + } + + test("trivial job cancellation") { + val rdd = makeRdd(1, Nil) + val jobId = submit(rdd, Array(0)) + cancel(jobId) + assert(failure.getMessage === s"Job $jobId cancelled") assertDataStructuresEmpty } @@ -323,6 +344,67 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assertDataStructuresEmpty } + test("run shuffle with map stage failure") { + val shuffleMapRdd = makeRdd(2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) + val reduceRdd = makeRdd(2, List(shuffleDep)) + submit(reduceRdd, Array(0, 1)) + + // Fail the map stage. This should cause the entire job to fail. + val stageFailureMessage = "Exception failure in map stage" + failed(taskSets(0), stageFailureMessage) + assert(failure.getMessage === s"Job aborted due to stage failure: $stageFailureMessage") + assertDataStructuresEmpty + } + + /** + * Makes sure that failures of stage used by multiple jobs are correctly handled. + * + * This test creates the following dependency graph: + * + * shuffleMapRdd1 shuffleMapRDD2 + * | \ | + * | \ | + * | \ | + * | \ | + * reduceRdd1 reduceRdd2 + * + * We start both shuffleMapRdds and then fail shuffleMapRdd1. As a result, the job listeners for + * reduceRdd1 and reduceRdd2 should both be informed that the job failed. shuffleMapRDD2 should + * also be cancelled, because it is only used by reduceRdd2 and reduceRdd2 cannot complete + * without shuffleMapRdd1. + */ + test("failure of stage used by two jobs") { + val shuffleMapRdd1 = makeRdd(2, Nil) + val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, null) + val shuffleMapRdd2 = makeRdd(2, Nil) + val shuffleDep2 = new ShuffleDependency(shuffleMapRdd2, null) + + val reduceRdd1 = makeRdd(2, List(shuffleDep1)) + val reduceRdd2 = makeRdd(2, List(shuffleDep1, shuffleDep2)) + + // We need to make our own listeners for this test, since by default submit uses the same + // listener for all jobs, and here we want to capture the failure for each job separately. + class FailureRecordingJobListener() extends JobListener { + var failureMessage: String = _ + override def taskSucceeded(index: Int, result: Any) {} + override def jobFailed(exception: Exception) = { failureMessage = exception.getMessage } + } + val listener1 = new FailureRecordingJobListener() + val listener2 = new FailureRecordingJobListener() + + submit(reduceRdd1, Array(0, 1), listener=listener1) + submit(reduceRdd2, Array(0, 1), listener=listener2) + + val stageFailureMessage = "Exception failure in map stage" + failed(taskSets(0), stageFailureMessage) + + assert(cancelledStages.contains(1)) + assert(listener1.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") + assert(listener2.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") + assertDataStructuresEmpty + } + test("run trivial shuffle with out-of-band failure and retry") { val shuffleMapRdd = makeRdd(2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) From 3bc054893bf2decdafa97a1e149e489ad154f066 Mon Sep 17 00:00:00 2001 From: Henry Saputra Date: Tue, 8 Apr 2014 14:23:16 -0700 Subject: [PATCH 027/641] Remove extra semicolon in import statement and unused import in ApplicationMaster Small nit cleanup to remove extra semicolon and unused import in Yarn's stable ApplicationMaster (it bothers me every time I saw it) Author: Henry Saputra Closes #358 from hsaputra/nitcleanup_removesemicolon_import_applicationmaster and squashes the following commits: bffb685 [Henry Saputra] Remove extra semicolon in import statement and unused import in ApplicationMaster.scala --- .../scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 30735cbfdf26e..c8a4d2e647cbd 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 @@ -18,7 +18,6 @@ package org.apache.spark.deploy.yarn import java.io.IOException -import java.net.Socket import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} @@ -36,7 +35,7 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} -import org.apache.hadoop.yarn.webapp.util.WebAppUtils; +import org.apache.hadoop.yarn.webapp.util.WebAppUtils import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil From a8d86b080ae26c96b078ba14dc60f3b528c07787 Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Tue, 8 Apr 2014 14:30:24 -0700 Subject: [PATCH 028/641] SPARK-1348 binding Master, Worker, and App Web UI to all interfaces Author: Kan Zhang Closes #318 from kanzhang/SPARK-1348 and squashes the following commits: e625a5f [Kan Zhang] reverting the changes to startJettyServer() 7a8084e [Kan Zhang] SPARK-1348 binding Master, Worker, and App Web UI to all interfaces --- .../scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala | 2 +- .../scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index bd75b2dfd0e07..01d9f52f4b7b4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -59,7 +59,7 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { def bind() { try { - serverInfo = Some(startJettyServer(host, port, handlers, master.conf)) + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, master.conf)) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index de76a5d5eb7bc..650f3da5ce3ff 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -60,7 +60,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I def bind() { try { - serverInfo = Some(JettyUtils.startJettyServer(host, port, handlers, worker.conf)) + serverInfo = Some(JettyUtils.startJettyServer("0.0.0.0", port, handlers, worker.conf)) logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index ef1ad872c8ef7..f53df7fbedf39 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -80,7 +80,7 @@ private[spark] class SparkUI( /** Bind the HTTP server which backs this web interface */ def bind() { try { - serverInfo = Some(startJettyServer(bindHost, port, handlers, sc.conf)) + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, sc.conf)) logInfo("Started Spark Web UI at http://%s:%d".format(publicHost, boundPort)) } catch { case e: Exception => From e25b593447a2e0aab9e5066f755e41be9068ecdc Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Tue, 8 Apr 2014 14:40:20 -0700 Subject: [PATCH 029/641] SPARK-1445: compute-classpath should not print error if lib_managed not found This was added to the check for the assembly jar, forgot it for the datanucleus jars. Author: Aaron Davidson Closes #361 from aarondav/cc and squashes the following commits: 8facc16 [Aaron Davidson] SPARK-1445: compute-classpath should not print error if lib_managed not found --- bin/compute-classpath.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index be37102dc069a..2a2bb376fd71f 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -63,7 +63,7 @@ fi # built with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark # assembly is built for Hive, before actually populating the CLASSPATH with the jars. # Note that this check order is faster (by up to half a second) in the case where Hive is not used. -num_datanucleus_jars=$(ls "$FWDIR"/lib_managed/jars/ | grep "datanucleus-.*\\.jar" | wc -l) +num_datanucleus_jars=$(ls "$FWDIR"/lib_managed/jars/ 2>/dev/null | grep "datanucleus-.*\\.jar" | wc -l) if [ $num_datanucleus_jars -gt 0 ]; then AN_ASSEMBLY_JAR=${ASSEMBLY_JAR:-$DEPS_ASSEMBLY_JAR} num_hive_files=$(jar tvf "$AN_ASSEMBLY_JAR" org/apache/hadoop/hive/ql/exec 2>/dev/null | wc -l) From fac6085cd774a4dba73ad1618537ef1817b2bcf3 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 8 Apr 2014 14:42:02 -0700 Subject: [PATCH 030/641] [SPARK-1397] Notify SparkListeners when stages fail or are cancelled. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit [I wanted to post this for folks to comment but it depends on (and thus includes the changes in) a currently outstanding PR, #305. You can look at just the second commit: https://github.com/kayousterhout/spark-1/commit/93f08baf731b9eaf5c9792a5373560526e2bccac to see just the changes relevant to this PR] Previously, when stages fail or get cancelled, the SparkListener is only notified indirectly through the SparkListenerJobEnd, where we sometimes pass in a single stage that failed. This worked before job cancellation, because jobs would only fail due to a single stage failure. However, with job cancellation, multiple running stages can fail when a job gets cancelled. Right now, this is not handled correctly, which results in stages that get stuck in the “Running Stages” window in the UI even though they’re dead. This PR changes the SparkListenerStageCompleted event to a SparkListenerStageEnded event, and uses this event to tell SparkListeners when stages fail in addition to when they complete successfully. This change is NOT publicly backward compatible for two reasons. First, it changes the SparkListener interface. We could alternately add a new event, SparkListenerStageFailed, and keep the existing SparkListenerStageCompleted. However, this is less consistent with the listener events for tasks / jobs ending, and will result in some code duplication for listeners (because failed and completed stages are handled in similar ways). Note that I haven’t finished updating the JSON code to correctly handle the new event because I’m waiting for feedback on whether this is a good or bad idea (hence the “WIP”). It is also not backwards compatible because it changes the publicly visible JobWaiter.jobFailed() method to no longer include a stage that caused the failure. I think this change should definitely stay, because with cancellation (as described above), a failure isn’t necessarily caused by a single stage. Author: Kay Ousterhout Closes #309 from kayousterhout/stage_cancellation and squashes the following commits: 5533ecd [Kay Ousterhout] Fixes in response to Mark's review 320c7c7 [Kay Ousterhout] Notify SparkListeners when stages fail or are cancelled. --- .../scala/org/apache/spark/FutureAction.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 121 +++++++++++------- .../apache/spark/scheduler/JobLogger.scala | 8 +- .../apache/spark/scheduler/JobResult.scala | 3 +- .../apache/spark/scheduler/JobWaiter.scala | 2 +- .../spark/scheduler/SparkListener.scala | 2 +- .../apache/spark/scheduler/StageInfo.scala | 9 ++ .../spark/ui/jobs/JobProgressListener.scala | 23 +--- .../org/apache/spark/util/JsonProtocol.scala | 11 +- .../spark/scheduler/DAGSchedulerSuite.scala | 45 ++++++- .../apache/spark/util/JsonProtocolSuite.scala | 3 +- 11 files changed, 151 insertions(+), 78 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index f2decd14ef6d9..2eec09cd1c795 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -141,7 +141,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: private def awaitResult(): Try[T] = { jobWaiter.awaitResult() match { case JobSucceeded => scala.util.Success(resultFunc) - case JobFailed(e: Exception, _) => scala.util.Failure(e) + case JobFailed(e: Exception) => scala.util.Failure(e) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c96d7435a7ed4..c41d6d75a1d49 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -342,22 +342,24 @@ class DAGScheduler( } /** - * Removes job and any stages that are not needed by any other job. Returns the set of ids for - * stages that were removed. The associated tasks for those stages need to be cancelled if we - * got here via job cancellation. + * Removes state for job and any stages that are not needed by any other job. Does not + * handle cancelling tasks or notifying the SparkListener about finished jobs/stages/tasks. + * + * @param job The job whose state to cleanup. + * @param resultStage Specifies the result stage for the job; if set to None, this method + * searches resultStagesToJob to find and cleanup the appropriate result stage. */ - private def removeJobAndIndependentStages(jobId: Int): Set[Int] = { - val registeredStages = jobIdToStageIds(jobId) - val independentStages = new HashSet[Int]() - if (registeredStages.isEmpty) { - logError("No stages registered for job " + jobId) + private def cleanupStateForJobAndIndependentStages(job: ActiveJob, resultStage: Option[Stage]) { + val registeredStages = jobIdToStageIds.get(job.jobId) + if (registeredStages.isEmpty || registeredStages.get.isEmpty) { + logError("No stages registered for job " + job.jobId) } else { - stageIdToJobIds.filterKeys(stageId => registeredStages.contains(stageId)).foreach { + stageIdToJobIds.filterKeys(stageId => registeredStages.get.contains(stageId)).foreach { case (stageId, jobSet) => - if (!jobSet.contains(jobId)) { + if (!jobSet.contains(job.jobId)) { logError( "Job %d not registered for stage %d even though that stage was registered for the job" - .format(jobId, stageId)) + .format(job.jobId, stageId)) } else { def removeStage(stageId: Int) { // data structures based on Stage @@ -394,23 +396,28 @@ class DAGScheduler( .format(stageId, stageIdToStage.size)) } - jobSet -= jobId + jobSet -= job.jobId if (jobSet.isEmpty) { // no other job needs this stage - independentStages += stageId removeStage(stageId) } } } } - independentStages.toSet - } + jobIdToStageIds -= job.jobId + jobIdToActiveJob -= job.jobId + activeJobs -= job - private def jobIdToStageIdsRemove(jobId: Int) { - if (!jobIdToStageIds.contains(jobId)) { - logDebug("Trying to remove unregistered job " + jobId) + if (resultStage.isEmpty) { + // Clean up result stages. + val resultStagesForJob = resultStageToJob.keySet.filter( + stage => resultStageToJob(stage).jobId == job.jobId) + if (resultStagesForJob.size != 1) { + logWarning( + s"${resultStagesForJob.size} result stages for job ${job.jobId} (expect exactly 1)") + } + resultStageToJob --= resultStagesForJob } else { - removeJobAndIndependentStages(jobId) - jobIdToStageIds -= jobId + resultStageToJob -= resultStage.get } } @@ -460,7 +467,7 @@ class DAGScheduler( val waiter = submitJob(rdd, func, partitions, callSite, allowLocal, resultHandler, properties) waiter.awaitResult() match { case JobSucceeded => {} - case JobFailed(exception: Exception, _) => + case JobFailed(exception: Exception) => logInfo("Failed to run " + callSite) throw exception } @@ -606,7 +613,16 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, -1))) + // Tell the listeners that all of the running stages have ended. Don't bother + // cancelling the stages because if the DAG scheduler is stopped, the entire application + // is in the process of getting stopped. + val stageFailedMessage = "Stage cancelled because SparkContext was shut down" + runningStages.foreach { stage => + val info = stageToInfos(stage) + info.stageFailed(stageFailedMessage) + listenerBus.post(SparkListenerStageCompleted(info)) + } + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) } return true } @@ -676,7 +692,7 @@ class DAGScheduler( } } catch { case e: Exception => - jobResult = JobFailed(e, job.finalStage.id) + jobResult = JobFailed(e) job.listener.jobFailed(e) } finally { val s = job.finalStage @@ -826,11 +842,8 @@ class DAGScheduler( job.numFinished += 1 // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { - jobIdToActiveJob -= stage.jobId - activeJobs -= job - resultStageToJob -= stage markStageAsFinished(stage) - jobIdToStageIdsRemove(job.jobId) + cleanupStateForJobAndIndependentStages(job, Some(stage)) listenerBus.post(SparkListenerJobEnd(job.jobId, JobSucceeded)) } job.listener.taskSucceeded(rt.outputId, event.result) @@ -982,7 +995,7 @@ class DAGScheduler( if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { - failJobAndIndependentStages(jobIdToActiveJob(jobId), s"Job $jobId cancelled") + failJobAndIndependentStages(jobIdToActiveJob(jobId), s"Job $jobId cancelled", None) } } @@ -999,7 +1012,8 @@ class DAGScheduler( stageToInfos(failedStage).completionTime = Some(System.currentTimeMillis()) for (resultStage <- dependentStages) { val job = resultStageToJob(resultStage) - failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason") + failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason", + Some(resultStage)) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") @@ -1008,28 +1022,45 @@ class DAGScheduler( /** * Fails a job and all stages that are only used by that job, and cleans up relevant state. + * + * @param resultStage The result stage for the job, if known. Used to cleanup state for the job + * slightly more efficiently than when not specified. */ - private def failJobAndIndependentStages(job: ActiveJob, failureReason: String) { + private def failJobAndIndependentStages(job: ActiveJob, failureReason: String, + resultStage: Option[Stage]) { val error = new SparkException(failureReason) job.listener.jobFailed(error) - // Cancel all tasks in independent stages. - val independentStages = removeJobAndIndependentStages(job.jobId) - independentStages.foreach(taskScheduler.cancelTasks) - - // Clean up remaining state we store for the job. - jobIdToActiveJob -= job.jobId - activeJobs -= job - jobIdToStageIds -= job.jobId - val resultStagesForJob = resultStageToJob.keySet.filter( - stage => resultStageToJob(stage).jobId == job.jobId) - if (resultStagesForJob.size != 1) { - logWarning( - s"${resultStagesForJob.size} result stages for job ${job.jobId} (expect exactly 1)") + // Cancel all independent, running stages. + val stages = jobIdToStageIds(job.jobId) + if (stages.isEmpty) { + logError("No stages registered for job " + job.jobId) } - resultStageToJob --= resultStagesForJob + stages.foreach { stageId => + val jobsForStage = stageIdToJobIds.get(stageId) + if (jobsForStage.isEmpty || !jobsForStage.get.contains(job.jobId)) { + logError( + "Job %d not registered for stage %d even though that stage was registered for the job" + .format(job.jobId, stageId)) + } else if (jobsForStage.get.size == 1) { + if (!stageIdToStage.contains(stageId)) { + logError("Missing Stage for stage with id $stageId") + } else { + // This is the only job that uses this stage, so fail the stage if it is running. + val stage = stageIdToStage(stageId) + if (runningStages.contains(stage)) { + taskScheduler.cancelTasks(stageId) + val stageInfo = stageToInfos(stage) + stageInfo.stageFailed(failureReason) + listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) + } + } + } + } + + cleanupStateForJobAndIndependentStages(job, resultStage) - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 5cecf9416b32c..7c5053998f1d6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -191,7 +191,11 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener */ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { val stageId = stageCompleted.stageInfo.stageId - stageLogInfo(stageId, "STAGE_ID=%d STATUS=COMPLETED".format(stageId)) + if (stageCompleted.stageInfo.failureReason.isEmpty) { + stageLogInfo(stageId, s"STAGE_ID=$stageId STATUS=COMPLETED") + } else { + stageLogInfo(stageId, s"STAGE_ID=$stageId STATUS=FAILED") + } } /** @@ -227,7 +231,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener var info = "JOB_ID=" + jobId jobEnd.jobResult match { case JobSucceeded => info += " STATUS=SUCCESS" - case JobFailed(exception, _) => + case JobFailed(exception) => info += " STATUS=FAILED REASON=" exception.getMessage.split("\\s+").foreach(info += _ + "_") case _ => diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index 3cf4e3077e4a4..047bd27056120 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -24,5 +24,4 @@ private[spark] sealed trait JobResult private[spark] case object JobSucceeded extends JobResult -// A failed stage ID of -1 means there is not a particular stage that caused the failure -private[spark] case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult +private[spark] case class JobFailed(exception: Exception) extends JobResult diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala index 8007b5418741e..e9bfee2248e5b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -64,7 +64,7 @@ private[spark] class JobWaiter[T]( override def jobFailed(exception: Exception): Unit = synchronized { _jobFinished = true - jobResult = JobFailed(exception, -1) + jobResult = JobFailed(exception) this.notifyAll() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index d4eb0ac88d8e8..d42e67742a4f7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -71,7 +71,7 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent */ trait SparkListener { /** - * Called when a stage is completed, with information on the completed stage + * Called when a stage completes successfully or fails, with information on the completed stage. */ def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { } diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 8115a7ed7896d..eec409b182ac6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -26,8 +26,17 @@ private[spark] class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None + /** Time when all tasks in the stage completed or when the stage was cancelled. */ var completionTime: Option[Long] = None + /** If the stage failed, the reason why. */ + var failureReason: Option[String] = None + var emittedTaskSizeWarning = false + + def stageFailed(reason: String) { + failureReason = Some(reason) + completionTime = Some(System.currentTimeMillis) + } } private[spark] diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 048f671c8788f..5167e20ea3d7d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -74,8 +74,13 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { // Remove by stageId, rather than by StageInfo, in case the StageInfo is from storage poolToActiveStages(stageIdToPool(stageId)).remove(stageId) activeStages.remove(stageId) - completedStages += stage - trimIfNecessary(completedStages) + if (stage.failureReason.isEmpty) { + completedStages += stage + trimIfNecessary(completedStages) + } else { + failedStages += stage + trimIfNecessary(failedStages) + } } /** If stages is too large, remove and garbage collect old stages */ @@ -215,20 +220,6 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { } } - override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { - jobEnd.jobResult match { - case JobFailed(_, stageId) => - activeStages.get(stageId).foreach { s => - // Remove by stageId, rather than by StageInfo, in case the StageInfo is from storage - activeStages.remove(s.stageId) - poolToActiveStages(stageIdToPool(stageId)).remove(s.stageId) - failedStages += s - trimIfNecessary(failedStages) - } - case _ => - } - } - override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { synchronized { val schedulingModeName = diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 2155a8888c85c..19654892bf661 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -166,12 +166,14 @@ private[spark] object JsonProtocol { val rddInfo = rddInfoToJson(stageInfo.rddInfo) val submissionTime = stageInfo.submissionTime.map(JInt(_)).getOrElse(JNothing) val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing) + val failureReason = stageInfo.failureReason.map(JString(_)).getOrElse(JNothing) ("Stage ID" -> stageInfo.stageId) ~ ("Stage Name" -> stageInfo.name) ~ ("Number of Tasks" -> stageInfo.numTasks) ~ ("RDD Info" -> rddInfo) ~ ("Submission Time" -> submissionTime) ~ ("Completion Time" -> completionTime) ~ + ("Failure Reason" -> failureReason) ~ ("Emitted Task Size Warning" -> stageInfo.emittedTaskSizeWarning) } @@ -259,9 +261,7 @@ private[spark] object JsonProtocol { val json = jobResult match { case JobSucceeded => Utils.emptyJson case jobFailed: JobFailed => - val exception = exceptionToJson(jobFailed.exception) - ("Exception" -> exception) ~ - ("Failed Stage ID" -> jobFailed.failedStageId) + JObject("Exception" -> exceptionToJson(jobFailed.exception)) } ("Result" -> result) ~ json } @@ -442,11 +442,13 @@ private[spark] object JsonProtocol { val rddInfo = rddInfoFromJson(json \ "RDD Info") val submissionTime = Utils.jsonOption(json \ "Submission Time").map(_.extract[Long]) val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]) + val failureReason = Utils.jsonOption(json \ "Failure Reason").map(_.extract[String]) val emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfo) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime + stageInfo.failureReason = failureReason stageInfo.emittedTaskSizeWarning = emittedTaskSizeWarning stageInfo } @@ -561,8 +563,7 @@ private[spark] object JsonProtocol { case `jobSucceeded` => JobSucceeded case `jobFailed` => val exception = exceptionFromJson(json \ "Exception") - val failedStageId = (json \ "Failed Stage ID").extract[Int] - new JobFailed(exception, failedStageId) + new JobFailed(exception) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 2e3026bffba2f..a74724d785ad3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -64,6 +64,21 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont override def defaultParallelism() = 2 } + /** Length of time to wait while draining listener events. */ + val WAIT_TIMEOUT_MILLIS = 10000 + val sparkListener = new SparkListener() { + val successfulStages = new HashSet[Int]() + val failedStages = new HashSet[Int]() + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { + val stageInfo = stageCompleted.stageInfo + if (stageInfo.failureReason.isEmpty) { + successfulStages += stageInfo.stageId + } else { + failedStages += stageInfo.stageId + } + } + } + var mapOutputTracker: MapOutputTrackerMaster = null var scheduler: DAGScheduler = null @@ -89,13 +104,16 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont /** The list of results that DAGScheduler has collected. */ val results = new HashMap[Int, Any]() var failure: Exception = _ - val listener = new JobListener() { + val jobListener = new JobListener() { override def taskSucceeded(index: Int, result: Any) = results.put(index, result) override def jobFailed(exception: Exception) = { failure = exception } } before { sc = new SparkContext("local", "DAGSchedulerSuite") + sparkListener.successfulStages.clear() + sparkListener.failedStages.clear() + sc.addSparkListener(sparkListener) taskSets.clear() cancelledStages.clear() cacheLocations.clear() @@ -187,7 +205,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont partitions: Array[Int], func: (TaskContext, Iterator[_]) => _ = jobComputeFunc, allowLocal: Boolean = false, - listener: JobListener = listener): Int = { + listener: JobListener = jobListener): Int = { val jobId = scheduler.nextJobId.getAndIncrement() runEvent(JobSubmitted(jobId, rdd, func, partitions, allowLocal, null, listener)) return jobId @@ -231,7 +249,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont override def toString = "DAGSchedulerSuite Local RDD" } val jobId = scheduler.nextJobId.getAndIncrement() - runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, null, listener)) + runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, null, jobListener)) assert(results === Map(0 -> 42)) assertDataStructuresEmpty } @@ -262,6 +280,9 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont submit(makeRdd(1, Nil), Array(0)) failed(taskSets(0), "some failure") assert(failure.getMessage === "Job aborted due to stage failure: some failure") + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(0)) + assert(sparkListener.failedStages.size === 1) assertDataStructuresEmpty } @@ -270,6 +291,9 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val jobId = submit(rdd, Array(0)) cancel(jobId) assert(failure.getMessage === s"Job $jobId cancelled") + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(0)) + assert(sparkListener.failedStages.size === 1) assertDataStructuresEmpty } @@ -354,6 +378,13 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val stageFailureMessage = "Exception failure in map stage" failed(taskSets(0), stageFailureMessage) assert(failure.getMessage === s"Job aborted due to stage failure: $stageFailureMessage") + + // Listener bus should get told about the map stage failing, but not the reduce stage + // (since the reduce stage hasn't been started yet). + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(1)) + assert(sparkListener.failedStages.size === 1) + assertDataStructuresEmpty } @@ -400,6 +431,14 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont failed(taskSets(0), stageFailureMessage) assert(cancelledStages.contains(1)) + + // Make sure the listeners got told about both failed stages. + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.successfulStages.isEmpty) + assert(sparkListener.failedStages.contains(1)) + assert(sparkListener.failedStages.contains(3)) + assert(sparkListener.failedStages.size === 2) + assert(listener1.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") assert(listener2.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") assertDataStructuresEmpty diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 7bab7da8fed68..0342a8aff3c28 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -89,7 +89,7 @@ class JsonProtocolSuite extends FunSuite { // JobResult val exception = new Exception("Out of Memory! Please restock film.") exception.setStackTrace(stackTrace) - val jobFailed = JobFailed(exception, 2) + val jobFailed = JobFailed(exception) testJobResult(JobSucceeded) testJobResult(jobFailed) @@ -294,7 +294,6 @@ class JsonProtocolSuite extends FunSuite { (result1, result2) match { case (JobSucceeded, JobSucceeded) => case (r1: JobFailed, r2: JobFailed) => - assert(r1.failedStageId === r2.failedStageId) assertEquals(r1.exception, r2.exception) case _ => fail("Job results don't match in types!") } From 12c077d5aa0b76a808a55db625c9677a52bd43f9 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Tue, 8 Apr 2014 16:19:22 -0700 Subject: [PATCH 031/641] SPARK-1433: Upgrade Mesos dependency to 0.17.0 Mesos 0.13.0 was released 6 months ago. Upgrade Mesos dependency to 0.17.0 Author: Sandeep Closes #355 from techaddict/mesos_update and squashes the following commits: f1abeee [Sandeep] SPARK-1433: Upgrade Mesos dependency to 0.17.0 Mesos 0.13.0 was released 6 months ago. Upgrade Mesos dependency to 0.17.0 --- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 6 ++++-- .../scheduler/cluster/mesos/MesosSchedulerBackend.scala | 2 +- docs/_config.yml | 2 +- pom.xml | 6 +++--- project/SparkBuild.scala | 2 +- 5 files changed, 10 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 06b041e1fd9a9..c478e685641d7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -194,10 +194,12 @@ private[spark] class CoarseMesosSchedulerBackend( .addResources(createResource("cpus", cpusToUse)) .addResources(createResource("mem", sc.executorMemory)) .build() - d.launchTasks(offer.getId, Collections.singletonList(task), filters) + d.launchTasks(Collections.singletonList(offer.getId), + Collections.singletonList(task), + filters) } else { // Filter it out - d.launchTasks(offer.getId, Collections.emptyList[MesosTaskInfo](), filters) + d.declineOffer(offer.getId, filters) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index dfdcafe19fb93..f878ae338fc95 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -223,7 +223,7 @@ private[spark] class MesosSchedulerBackend( // Reply to the offers val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? for (i <- 0 until offers.size) { - d.launchTasks(offers(i).getId, mesosTasks(i), filters) + d.launchTasks(Collections.singletonList(offers(i).getId), mesosTasks(i), filters) } } } finally { diff --git a/docs/_config.yml b/docs/_config.yml index d585b8c5ea763..bd5ed6c9220d2 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -7,6 +7,6 @@ SPARK_VERSION: 1.0.0-SNAPSHOT SPARK_VERSION_SHORT: 1.0.0 SCALA_BINARY_VERSION: "2.10" SCALA_VERSION: "2.10.4" -MESOS_VERSION: 0.13.0 +MESOS_VERSION: 0.17.0 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/pom.xml b/pom.xml index c03bb35c99442..11511bcb9da52 100644 --- a/pom.xml +++ b/pom.xml @@ -112,7 +112,7 @@ 2.10.4 2.10 - 0.13.0 + 0.17.0 org.spark-project.akka 2.2.3-shaded-protobuf 1.7.5 @@ -848,7 +848,7 @@ - + hadoop-provided @@ -893,6 +893,6 @@ - + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 6b8740d9f21a1..08667aac2cd2d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -332,7 +332,7 @@ object SparkBuild extends Build { "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", - "org.apache.mesos" % "mesos" % "0.13.0", + "org.apache.mesos" % "mesos" % "0.17.0", "commons-net" % "commons-net" % "2.2", "net.java.dev.jets3t" % "jets3t" % "0.7.1" excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", From ce8ec5456169682f27f846e7b8d51e6c4bcf75e3 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 8 Apr 2014 18:15:52 -0700 Subject: [PATCH 032/641] Spark 1271: Co-Group and Group-By should pass Iterable[X] Author: Holden Karau Closes #242 from holdenk/spark-1320-cogroupandgroupshouldpassiterator and squashes the following commits: f289536 [Holden Karau] Fix bad merge, should have been Iterable rather than Iterator 77048f8 [Holden Karau] Fix merge up to master d3fe909 [Holden Karau] use toSeq instead 7a092a3 [Holden Karau] switch resultitr to resultiterable eb06216 [Holden Karau] maybe I should have had a coffee first. use correct import for guava iterables c5075aa [Holden Karau] If guava 14 had iterables 2d06e10 [Holden Karau] Fix Java 8 cogroup tests for the new API 11e730c [Holden Karau] Fix streaming tests 66b583d [Holden Karau] Fix the core test suite to compile 4ed579b [Holden Karau] Refactor from iterator to iterable d052c07 [Holden Karau] Python tests now pass with iterator pandas 3bcd81d [Holden Karau] Revert "Try and make pickling list iterators work" cd1e81c [Holden Karau] Try and make pickling list iterators work c60233a [Holden Karau] Start investigating moving to iterators for python API like the Java/Scala one. tl;dr: We will have to write our own iterator since the default one doesn't pickle well 88a5cef [Holden Karau] Fix cogroup test in JavaAPISuite for streaming a5ee714 [Holden Karau] oops, was checking wrong iterator e687f21 [Holden Karau] Fix groupbykey test in JavaAPISuite of streaming ec8cc3e [Holden Karau] Fix test issues\! 4b0eeb9 [Holden Karau] Switch cast in PairDStreamFunctions fa395c9 [Holden Karau] Revert "Add a join based on the problem in SVD" ec99e32 [Holden Karau] Revert "Revert this but for now put things in list pandas" b692868 [Holden Karau] Revert 7e533f7 [Holden Karau] Fix the bug 8a5153a [Holden Karau] Revert me, but we have some stuff to debug b4e86a9 [Holden Karau] Add a join based on the problem in SVD c4510e2 [Holden Karau] Revert this but for now put things in list pandas b4e0b1d [Holden Karau] Fix style issues 71e8b9f [Holden Karau] I really need to stop calling size on iterators, it is the path of sadness. b1ae51a [Holden Karau] Fix some of the types in the streaming JavaAPI suite. Probably still needs more work 37888ec [Holden Karau] core/tests now pass 249abde [Holden Karau] org.apache.spark.rdd.PairRDDFunctionsSuite passes 6698186 [Holden Karau] Revert "I think this might be a bad rabbit hole. Started work to make CoGroupedRDD use iterator and then went crazy" fe992fe [Holden Karau] hmmm try and fix up basic operation suite 172705c [Holden Karau] Fix Java API suite caafa63 [Holden Karau] I think this might be a bad rabbit hole. Started work to make CoGroupedRDD use iterator and then went crazy 88b3329 [Holden Karau] Fix groupbykey to actually give back an iterator 4991af6 [Holden Karau] Fix some tests be50246 [Holden Karau] Calling size on an iterator is not so good if we want to use it after 687ffbc [Holden Karau] This is the it compiles point of replacing Seq with Iterator and JList with JIterator in the groupby and cogroup signatures --- .../scala/org/apache/spark/bagel/Bagel.scala | 20 ++++--- .../apache/spark/api/java/JavaPairRDD.scala | 36 ++++++------ .../apache/spark/api/java/JavaRDDLike.scala | 6 +- .../apache/spark/rdd/PairRDDFunctions.scala | 39 +++++++------ .../main/scala/org/apache/spark/rdd/RDD.scala | 6 +- .../java/org/apache/spark/JavaAPISuite.java | 20 ++++--- .../scala/org/apache/spark/FailureSuite.scala | 4 +- .../org/apache/spark/PipedRDDSuite.scala | 2 +- .../spark/rdd/PairRDDFunctionsSuite.scala | 12 ++-- .../ExternalAppendOnlyMapSuite.scala | 4 +- .../apache/spark/examples/JavaPageRank.java | 21 ++++--- .../bagel/WikipediaPageRankStandalone.scala | 14 +++-- .../java/org/apache/spark/Java8APISuite.java | 11 ++-- .../org/apache/spark/mllib/linalg/SVD.scala | 6 +- .../spark/mllib/recommendation/ALS.scala | 4 +- .../org/apache/spark/mllib/util/LAUtils.scala | 6 +- python/pyspark/join.py | 5 +- python/pyspark/rdd.py | 10 ++-- python/pyspark/resultiterable.py | 33 +++++++++++ .../streaming/api/java/JavaPairDStream.scala | 42 +++++++------- .../dstream/PairDStreamFunctions.scala | 29 +++++----- .../streaming/dstream/StateDStream.scala | 13 +++-- .../apache/spark/streaming/JavaAPISuite.java | 58 ++++++++++++++++--- .../streaming/BasicOperationsSuite.scala | 4 +- 24 files changed, 252 insertions(+), 153 deletions(-) create mode 100644 python/pyspark/resultiterable.py diff --git a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala index 70c7474a936dc..70a99b33d753c 100644 --- a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala @@ -220,20 +220,23 @@ object Bagel extends Logging { */ private def comp[K: Manifest, V <: Vertex, M <: Message[K], C]( sc: SparkContext, - grouped: RDD[(K, (Seq[C], Seq[V]))], + grouped: RDD[(K, (Iterable[C], Iterable[V]))], compute: (V, Option[C]) => (V, Array[M]), storageLevel: StorageLevel ): (RDD[(K, (V, Array[M]))], Int, Int) = { var numMsgs = sc.accumulator(0) var numActiveVerts = sc.accumulator(0) - val processed = grouped.flatMapValues { - case (_, vs) if vs.size == 0 => None - case (c, vs) => + val processed = grouped.mapValues(x => (x._1.iterator, x._2.iterator)) + .flatMapValues { + case (_, vs) if !vs.hasNext => None + case (c, vs) => { val (newVert, newMsgs) = - compute(vs(0), c match { - case Seq(comb) => Some(comb) - case Seq() => None - }) + compute(vs.next, + c.hasNext match { + case true => Some(c.next) + case false => None + } + ) numMsgs += newMsgs.size if (newVert.active) { @@ -241,6 +244,7 @@ object Bagel extends Logging { } Some((newVert, newMsgs)) + } }.persist(storageLevel) // Force evaluation of processed RDD for accurate performance measurements diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 9596dbaf75488..e6c5d85917678 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.api.java import java.util.{Comparator, List => JList} +import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -250,14 +251,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. */ - def groupByKey(partitioner: Partitioner): JavaPairRDD[K, JList[V]] = + def groupByKey(partitioner: Partitioner): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(partitioner))) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): JavaPairRDD[K, JList[V]] = + def groupByKey(numPartitions: Int): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(numPartitions))) /** @@ -367,7 +368,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. */ - def groupByKey(): JavaPairRDD[K, JList[V]] = + def groupByKey(): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey())) /** @@ -462,7 +463,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: JavaPairRDD[K, W], partitioner: Partitioner) - : JavaPairRDD[K, (JList[V], JList[W])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other, partitioner))) /** @@ -470,14 +471,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], - partitioner: Partitioner): JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + partitioner: Partitioner): JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, partitioner))) /** * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = + def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other))) /** @@ -485,7 +486,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2))) /** @@ -493,7 +494,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int) - : JavaPairRDD[K, (JList[V], JList[W])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) /** @@ -501,16 +502,16 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], numPartitions: Int) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numPartitions))) /** Alias for cogroup. */ - def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = + def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.groupWith(other))) /** Alias for cogroup. */ def groupWith[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.groupWith(other1, other2))) /** @@ -695,21 +696,22 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) object JavaPairRDD { private[spark] - def groupByResultToJava[K: ClassTag, T](rdd: RDD[(K, Seq[T])]): RDD[(K, JList[T])] = { - rddToPairRDDFunctions(rdd).mapValues(seqAsJavaList) + def groupByResultToJava[K: ClassTag, T](rdd: RDD[(K, Iterable[T])]): RDD[(K, JIterable[T])] = { + rddToPairRDDFunctions(rdd).mapValues(asJavaIterable) } private[spark] def cogroupResultToJava[K: ClassTag, V, W]( - rdd: RDD[(K, (Seq[V], Seq[W]))]): RDD[(K, (JList[V], JList[W]))] = { - rddToPairRDDFunctions(rdd).mapValues(x => (seqAsJavaList(x._1), seqAsJavaList(x._2))) + rdd: RDD[(K, (Iterable[V], Iterable[W]))]): RDD[(K, (JIterable[V], JIterable[W]))] = { + rddToPairRDDFunctions(rdd).mapValues(x => (asJavaIterable(x._1), asJavaIterable(x._2))) } private[spark] def cogroupResult2ToJava[K: ClassTag, V, W1, W2]( - rdd: RDD[(K, (Seq[V], Seq[W1], Seq[W2]))]): RDD[(K, (JList[V], JList[W1], JList[W2]))] = { + rdd: RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))]) + : RDD[(K, (JIterable[V], JIterable[W1], JIterable[W2]))] = { rddToPairRDDFunctions(rdd) - .mapValues(x => (seqAsJavaList(x._1), seqAsJavaList(x._2), seqAsJavaList(x._3))) + .mapValues(x => (asJavaIterable(x._1), asJavaIterable(x._2), asJavaIterable(x._3))) } def fromRDD[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]): JavaPairRDD[K, V] = { 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 6e8ec8e0c7629..ae577b500ccb4 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 @@ -17,7 +17,7 @@ package org.apache.spark.api.java -import java.util.{Comparator, Iterator => JIterator, List => JList} +import java.util.{Comparator, List => JList, Iterator => JIterator} import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ @@ -204,7 +204,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JList[T]] = { + def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JIterable[T]] = { implicit val ctagK: ClassTag[K] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(fakeClassTag))) @@ -214,7 +214,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JList[T]] = { + def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JIterable[T]] = { implicit val ctagK: ClassTag[K] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(fakeClassTag[K]))) 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 14386ff5b9127..a92a84b5342d1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -261,7 +261,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. */ - def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = { + def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = { // groupByKey shouldn't use map side combine because map side combine does not // reduce the amount of data shuffled and requires all map side data be inserted // into a hash table, leading to more objects in the old gen. @@ -270,14 +270,14 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def mergeCombiners(c1: ArrayBuffer[V], c2: ArrayBuffer[V]) = c1 ++ c2 val bufs = combineByKey[ArrayBuffer[V]]( createCombiner _, mergeValue _, mergeCombiners _, partitioner, mapSideCombine=false) - bufs.asInstanceOf[RDD[(K, Seq[V])]] + bufs.mapValues(_.toIterable) } /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): RDD[(K, Seq[V])] = { + def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = { groupByKey(new HashPartitioner(numPartitions)) } @@ -298,7 +298,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + for (v <- vs; w <- ws) yield (v, w) } } @@ -311,9 +311,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (ws.isEmpty) { - vs.iterator.map(v => (v, None)) + vs.map(v => (v, None)) } else { - for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) + for (v <- vs; w <- ws) yield (v, Some(w)) } } } @@ -328,9 +328,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) : RDD[(K, (Option[V], W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (vs.isEmpty) { - ws.iterator.map(w => (None, w)) + ws.map(w => (None, w)) } else { - for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) + for (v <- vs; w <- ws) yield (Some(v), w) } } } @@ -358,7 +358,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. */ - def groupByKey(): RDD[(K, Seq[V])] = { + def groupByKey(): RDD[(K, Iterable[V])] = { groupByKey(defaultPartitioner(self)) } @@ -453,7 +453,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner) + : RDD[(K, (Iterable[V], Iterable[W]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } @@ -468,13 +469,15 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) cg.mapValues { case Seq(vs, w1s, w2s) => - (vs.asInstanceOf[Seq[V]], w1s.asInstanceOf[Seq[W1]], w2s.asInstanceOf[Seq[W2]]) + (vs.asInstanceOf[Seq[V]], + w1s.asInstanceOf[Seq[W1]], + w2s.asInstanceOf[Seq[W2]]) } } @@ -482,7 +485,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner(self, other)) } @@ -491,7 +494,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } @@ -499,7 +502,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, new HashPartitioner(numPartitions)) } @@ -508,18 +511,18 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { cogroup(other1, other2, new HashPartitioner(numPartitions)) } /** Alias for cogroup. */ - def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner(self, other)) } /** Alias for cogroup. */ def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } 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 bf3c57ad41eb2..74fa2a4fcd401 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -438,20 +438,20 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD of grouped items. */ - def groupBy[K: ClassTag](f: T => K): RDD[(K, Seq[T])] = + def groupBy[K: ClassTag](f: T => K): RDD[(K, Iterable[T])] = groupBy[K](f, defaultPartitioner(this)) /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] = + def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Iterable[T])] = groupBy(f, new HashPartitioner(numPartitions)) /** * Return an RDD of grouped items. */ - def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Seq[T])] = { + def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Iterable[T])] = { val cleanF = sc.clean(f) this.map(t => (cleanF(t), t)).groupByKey(p) } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 762405be2a8f9..ab2fdac553349 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -18,10 +18,12 @@ package org.apache.spark; import java.io.*; +import java.lang.StringBuilder; import java.util.*; import scala.Tuple2; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.base.Optional; import com.google.common.base.Charsets; @@ -197,7 +199,7 @@ public void lookup() { new Tuple2("Oranges", "Citrus") )); Assert.assertEquals(2, categories.lookup("Oranges").size()); - Assert.assertEquals(2, categories.groupByKey().lookup("Oranges").get(0).size()); + Assert.assertEquals(2, Iterables.size(categories.groupByKey().lookup("Oranges").get(0))); } @Test @@ -209,15 +211,15 @@ public Boolean call(Integer x) { return x % 2 == 0; } }; - JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); + JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds oddsAndEvens = rdd.groupBy(isOdd, 1); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds } @SuppressWarnings("unchecked") @@ -232,9 +234,9 @@ public void cogroup() { new Tuple2("Oranges", 2), new Tuple2("Apples", 3) )); - JavaPairRDD, List>> cogrouped = categories.cogroup(prices); - Assert.assertEquals("[Fruit, Citrus]", cogrouped.lookup("Oranges").get(0)._1().toString()); - Assert.assertEquals("[2]", cogrouped.lookup("Oranges").get(0)._2().toString()); + JavaPairRDD, Iterable>> cogrouped = categories.cogroup(prices); + Assert.assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + Assert.assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); cogrouped.collect(); } diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index f3fb64d87a2fd..12dbebcb28644 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -72,7 +72,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { throw new Exception("Intentional task failure") } } - (k, v(0) * v(0)) + (k, v.head * v.head) }.collect() FailureSuiteState.synchronized { assert(FailureSuiteState.tasksRun === 4) @@ -137,5 +137,3 @@ class FailureSuite extends FunSuite with LocalSparkContext { // TODO: Need to add tests with shuffle fetch failures. } - - diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index 627e9b5cd9060..867b28cc0d971 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -85,7 +85,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { (f: String => Unit) => { bl.value.map(f(_)); f("\u0001") }, - (i: Tuple2[String, Seq[String]], f: String => Unit) => { + (i: Tuple2[String, Iterable[String]], f: String => Unit) => { for (e <- i._2) { f(e + "_") } 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 f9e994b13dfbc..8f3e6bd21b752 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -225,11 +225,12 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.groupWith(rdd2).collect() assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (ArrayBuffer(1, 2), ArrayBuffer('x'))), - (2, (ArrayBuffer(1), ArrayBuffer('y', 'z'))), - (3, (ArrayBuffer(1), ArrayBuffer())), - (4, (ArrayBuffer(), ArrayBuffer('w'))) + val joinedSet = joined.map(x => (x._1, (x._2._1.toList, x._2._2.toList))).toSet + assert(joinedSet === Set( + (1, (List(1, 2), List('x'))), + (2, (List(1), List('y', 'z'))), + (3, (List(1), List())), + (4, (List(), List('w'))) )) } @@ -447,4 +448,3 @@ class ConfigTestFormat() extends FakeFormat() with Configurable { super.getRecordWriter(p1) } } - diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index fce1184d46364..cdebefb67510c 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -174,9 +174,9 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { assert(result1.toSet == Set[(Int, Int)]((0, 5), (1, 5))) // groupByKey - val result2 = rdd.groupByKey().collect() + val result2 = rdd.groupByKey().collect().map(x => (x._1, x._2.toList)).toSet assert(result2.toSet == Set[(Int, Seq[Int])] - ((0, ArrayBuffer[Int](1, 1, 1, 1, 1)), (1, ArrayBuffer[Int](1, 1, 1, 1, 1)))) + ((0, List[Int](1, 1, 1, 1, 1)), (1, List[Int](1, 1, 1, 1, 1)))) } test("simple cogroup") { 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 eb70fb547564c..8513ba07e7705 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -17,7 +17,10 @@ package org.apache.spark.examples; + import scala.Tuple2; + +import com.google.common.collect.Iterables; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -26,8 +29,9 @@ import org.apache.spark.api.java.function.PairFlatMapFunction; import org.apache.spark.api.java.function.PairFunction; -import java.util.List; import java.util.ArrayList; +import java.util.List; +import java.util.Iterator; import java.util.regex.Pattern; /** @@ -66,7 +70,7 @@ public static void main(String[] args) throws Exception { JavaRDD lines = ctx.textFile(args[1], 1); // Loads all URLs from input file and initialize their neighbors. - JavaPairRDD> links = lines.mapToPair(new PairFunction() { + JavaPairRDD> links = lines.mapToPair(new PairFunction() { @Override public Tuple2 call(String s) { String[] parts = SPACES.split(s); @@ -75,9 +79,9 @@ public Tuple2 call(String s) { }).distinct().groupByKey().cache(); // Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. - JavaPairRDD ranks = links.mapValues(new Function, Double>() { + JavaPairRDD ranks = links.mapValues(new Function, Double>() { @Override - public Double call(List rs) { + public Double call(Iterable rs) { return 1.0; } }); @@ -86,12 +90,13 @@ public Double call(List rs) { for (int current = 0; current < Integer.parseInt(args[2]); current++) { // Calculates URL contributions to the rank of other URLs. JavaPairRDD contribs = links.join(ranks).values() - .flatMapToPair(new PairFlatMapFunction, Double>, String, Double>() { + .flatMapToPair(new PairFlatMapFunction, Double>, String, Double>() { @Override - public Iterable> call(Tuple2, Double> s) { + public Iterable> call(Tuple2, Double> s) { + int urlCount = Iterables.size(s._1); 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() / urlCount)); } return results; } diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index 27afa6b642758..7aac6a13597e6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -115,12 +115,16 @@ object WikipediaPageRankStandalone { var ranks = links.mapValues { edges => defaultRank } for (i <- 1 to numIterations) { val contribs = links.groupWith(ranks).flatMap { - case (id, (linksWrapper, rankWrapper)) => - if (linksWrapper.length > 0) { - if (rankWrapper.length > 0) { - linksWrapper(0).map(dest => (dest, rankWrapper(0) / linksWrapper(0).size)) + case (id, (linksWrapperIterable, rankWrapperIterable)) => + val linksWrapper = linksWrapperIterable.iterator + val rankWrapper = rankWrapperIterable.iterator + if (linksWrapper.hasNext) { + val linksWrapperHead = linksWrapper.next + if (rankWrapper.hasNext) { + val rankWrapperHead = rankWrapper.next + linksWrapperHead.map(dest => (dest, rankWrapperHead / linksWrapperHead.size)) } else { - linksWrapper(0).map(dest => (dest, defaultRank / linksWrapper(0).size)) + linksWrapperHead.map(dest => (dest, defaultRank / linksWrapperHead.size)) } } else { Array[(String, Double)]() diff --git a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java index f67251217ed4a..7eb8b45fc3cf0 100644 --- a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java +++ b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java @@ -23,6 +23,7 @@ import scala.Tuple2; +import com.google.common.collections.Iterables; import com.google.common.base.Optional; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; @@ -85,15 +86,15 @@ public void foreach() { public void groupBy() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); Function isOdd = x -> x % 2 == 0; - JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); + JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds oddsAndEvens = rdd.groupBy(isOdd, 1); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds } @Test 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 3e7cc648d1d37..0d97b7d92f155 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 @@ -69,11 +69,11 @@ class SVD { /** * Compute SVD using the current set parameters - * Returns (U, S, V) such that A = USV^T + * Returns (U, S, V) such that A = USV^T * U is a row-by-row dense matrix * S is a simple double array of singular values * V is a 2d array matrix - * See [[denseSVD]] for more documentation + * See [[denseSVD]] for more documentation */ def compute(matrix: RDD[Array[Double]]): (RDD[Array[Double]], Array[Double], Array[Array[Double]]) = { @@ -393,5 +393,3 @@ object SVD { System.exit(0) } } - - 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 0cc9f48769f83..3124fac326d22 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 @@ -421,12 +421,12 @@ class ALS private ( * Compute the new feature vectors for a block of the users matrix given the list of factors * it received from each product and its InLinkBlock. */ - private def updateBlock(messages: Seq[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock, + private def updateBlock(messages: Iterable[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock, rank: Int, lambda: Double, alpha: Double, YtY: Option[Broadcast[DoubleMatrix]]) : Array[Array[Double]] = { // Sort the incoming block factor messages by block ID and make them an array - val blockFactors = messages.sortBy(_._1).map(_._2).toArray // Array[Array[Double]] + val blockFactors = messages.toSeq.sortBy(_._1).map(_._2).toArray // Array[Array[Double]] val numBlocks = blockFactors.length val numUsers = inLinkBlock.elementIds.length diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala index afe081295bfae..87aac347579c7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala @@ -38,8 +38,10 @@ object LAUtils { case (i, cols) => val rowArray = Array.ofDim[Double](n) var j = 0 - while (j < cols.size) { - rowArray(cols(j)._1) = cols(j)._2 + val colsItr = cols.iterator + while (colsItr.hasNext) { + val element = colsItr.next + rowArray(element._1) = element._2 j += 1 } MatrixRow(i, rowArray) diff --git a/python/pyspark/join.py b/python/pyspark/join.py index 5f4294fb1b777..6f94d26ef86a9 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -31,11 +31,12 @@ OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. """ +from pyspark.resultiterable import ResultIterable def _do_python_join(rdd, other, numPartitions, dispatch): vs = rdd.map(lambda (k, v): (k, (1, v))) ws = other.map(lambda (k, v): (k, (2, v))) - return vs.union(ws).groupByKey(numPartitions).flatMapValues(dispatch) + return vs.union(ws).groupByKey(numPartitions).flatMapValues(lambda x : dispatch(x.__iter__())) def python_join(rdd, other, numPartitions): @@ -88,5 +89,5 @@ def dispatch(seq): vbuf.append(v) elif n == 2: wbuf.append(v) - return (vbuf, wbuf) + return (ResultIterable(vbuf), ResultIterable(wbuf)) return vs.union(ws).groupByKey(numPartitions).mapValues(dispatch) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index fb27863e07f55..91fc7e637e2c6 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -38,6 +38,7 @@ from pyspark.statcounter import StatCounter from pyspark.rddsampler import RDDSampler from pyspark.storagelevel import StorageLevel +from pyspark.resultiterable import ResultIterable from py4j.java_collections import ListConverter, MapConverter @@ -1118,7 +1119,7 @@ def groupByKey(self, numPartitions=None): Hash-partitions the resulting RDD with into numPartitions partitions. >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) - >>> sorted(x.groupByKey().collect()) + >>> map((lambda (x,y): (x, list(y))), sorted(x.groupByKey().collect())) [('a', [1, 1]), ('b', [1])] """ @@ -1133,7 +1134,7 @@ def mergeCombiners(a, b): return a + b return self.combineByKey(createCombiner, mergeValue, mergeCombiners, - numPartitions) + numPartitions).mapValues(lambda x: ResultIterable(x)) # TODO: add tests def flatMapValues(self, f): @@ -1180,7 +1181,7 @@ def cogroup(self, other, numPartitions=None): >>> x = sc.parallelize([("a", 1), ("b", 4)]) >>> y = sc.parallelize([("a", 2)]) - >>> sorted(x.cogroup(y).collect()) + >>> map((lambda (x,y): (x, (list(y[0]), list(y[1])))), sorted(list(x.cogroup(y).collect()))) [('a', ([1], [2])), ('b', ([4], []))] """ return python_cogroup(self, other, numPartitions) @@ -1217,7 +1218,7 @@ def keyBy(self, f): >>> x = sc.parallelize(range(0,3)).keyBy(lambda x: x*x) >>> y = sc.parallelize(zip(range(0,5), range(0,5))) - >>> sorted(x.cogroup(y).collect()) + >>> map((lambda (x,y): (x, (list(y[0]), (list(y[1]))))), sorted(x.cogroup(y).collect())) [(0, ([0], [0])), (1, ([1], [1])), (2, ([], [2])), (3, ([], [3])), (4, ([2], [4]))] """ return self.map(lambda x: (f(x), x)) @@ -1317,7 +1318,6 @@ def getStorageLevel(self): # keys in the pairs. This could be an expensive operation, since those # hashes aren't retained. - class PipelinedRDD(RDD): """ Pipelined maps: diff --git a/python/pyspark/resultiterable.py b/python/pyspark/resultiterable.py new file mode 100644 index 0000000000000..7f418f8d2e29a --- /dev/null +++ b/python/pyspark/resultiterable.py @@ -0,0 +1,33 @@ +# +# 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. +# + +__all__ = ["ResultIterable"] + +import collections + +class ResultIterable(collections.Iterable): + """ + A special result iterable. This is used because the standard iterator can not be pickled + """ + def __init__(self, data): + self.data = data + self.index = 0 + self.maxindex = len(data) + def __iter__(self): + return iter(self.data) + def __len__(self): + return len(self.data) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index ac451d1913aaa..2ac943d7bf781 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.api.java -import java.lang.{Long => JLong} +import java.lang.{Long => JLong, Iterable => JIterable} import java.util.{List => JList} import scala.collection.JavaConversions._ @@ -115,15 +115,15 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. */ - def groupByKey(): JavaPairDStream[K, JList[V]] = - dstream.groupByKey().mapValues(seqAsJavaList _) + def groupByKey(): JavaPairDStream[K, JIterable[V]] = + dstream.groupByKey().mapValues(asJavaIterable _) /** * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): JavaPairDStream[K, JList[V]] = - dstream.groupByKey(numPartitions).mapValues(seqAsJavaList _) + def groupByKey(numPartitions: Int): JavaPairDStream[K, JIterable[V]] = + dstream.groupByKey(numPartitions).mapValues(asJavaIterable _) /** * Return a new DStream by applying `groupByKey` on each RDD of `this` DStream. @@ -131,8 +131,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * single sequence to generate the RDDs of the new DStream. org.apache.spark.Partitioner * is used to control the partitioning of each RDD. */ - def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JList[V]] = - dstream.groupByKey(partitioner).mapValues(seqAsJavaList _) + def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JIterable[V]] = + dstream.groupByKey(partitioner).mapValues(asJavaIterable _) /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are @@ -196,8 +196,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval */ - def groupByKeyAndWindow(windowDuration: Duration): JavaPairDStream[K, JList[V]] = { - dstream.groupByKeyAndWindow(windowDuration).mapValues(seqAsJavaList _) + def groupByKeyAndWindow(windowDuration: Duration): JavaPairDStream[K, JIterable[V]] = { + dstream.groupByKeyAndWindow(windowDuration).mapValues(asJavaIterable _) } /** @@ -211,8 +211,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * DStream's batching interval */ def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration) - : JavaPairDStream[K, JList[V]] = { - dstream.groupByKeyAndWindow(windowDuration, slideDuration).mapValues(seqAsJavaList _) + : JavaPairDStream[K, JIterable[V]] = { + dstream.groupByKeyAndWindow(windowDuration, slideDuration).mapValues(asJavaIterable _) } /** @@ -227,9 +227,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param numPartitions Number of partitions of each RDD in the new DStream. */ def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int) - :JavaPairDStream[K, JList[V]] = { + :JavaPairDStream[K, JIterable[V]] = { dstream.groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) - .mapValues(seqAsJavaList _) + .mapValues(asJavaIterable _) } /** @@ -247,9 +247,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( windowDuration: Duration, slideDuration: Duration, partitioner: Partitioner - ):JavaPairDStream[K, JList[V]] = { + ):JavaPairDStream[K, JIterable[V]] = { dstream.groupByKeyAndWindow(windowDuration, slideDuration, partitioner) - .mapValues(seqAsJavaList _) + .mapValues(asJavaIterable _) } /** @@ -518,9 +518,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Hash partitioning is used to generate the RDDs with Spark's default number * of partitions. */ - def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JList[V], JList[W])] = { + def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JIterable[V], JIterable[W])] = { implicit val cm: ClassTag[W] = fakeClassTag - dstream.cogroup(other.dstream).mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) + dstream.cogroup(other.dstream).mapValues(t => (asJavaIterable(t._1), asJavaIterable((t._2)))) } /** @@ -530,10 +530,10 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( def cogroup[W]( other: JavaPairDStream[K, W], numPartitions: Int - ): JavaPairDStream[K, (JList[V], JList[W])] = { + ): JavaPairDStream[K, (JIterable[V], JIterable[W])] = { implicit val cm: ClassTag[W] = fakeClassTag dstream.cogroup(other.dstream, numPartitions) - .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) + .mapValues(t => (asJavaIterable(t._1), asJavaIterable((t._2)))) } /** @@ -543,10 +543,10 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( def cogroup[W]( other: JavaPairDStream[K, W], partitioner: Partitioner - ): JavaPairDStream[K, (JList[V], JList[W])] = { + ): JavaPairDStream[K, (JIterable[V], JIterable[W])] = { implicit val cm: ClassTag[W] = fakeClassTag dstream.cogroup(other.dstream, partitioner) - .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) + .mapValues(t => (asJavaIterable(t._1), asJavaIterable((t._2)))) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index 2473496949360..354bc132dcdc0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -51,7 +51,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. */ - def groupByKey(): DStream[(K, Seq[V])] = { + def groupByKey(): DStream[(K, Iterable[V])] = { groupByKey(defaultPartitioner()) } @@ -59,7 +59,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): DStream[(K, Seq[V])] = { + def groupByKey(numPartitions: Int): DStream[(K, Iterable[V])] = { groupByKey(defaultPartitioner(numPartitions)) } @@ -67,12 +67,12 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Return a new DStream by applying `groupByKey` on each RDD. The supplied * org.apache.spark.Partitioner is used to control the partitioning of each RDD. */ - def groupByKey(partitioner: Partitioner): DStream[(K, Seq[V])] = { + def groupByKey(partitioner: Partitioner): DStream[(K, Iterable[V])] = { val createCombiner = (v: V) => ArrayBuffer[V](v) val mergeValue = (c: ArrayBuffer[V], v: V) => (c += v) val mergeCombiner = (c1: ArrayBuffer[V], c2: ArrayBuffer[V]) => (c1 ++ c2) combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner) - .asInstanceOf[DStream[(K, Seq[V])]] + .asInstanceOf[DStream[(K, Iterable[V])]] } /** @@ -126,7 +126,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval */ - def groupByKeyAndWindow(windowDuration: Duration): DStream[(K, Seq[V])] = { + def groupByKeyAndWindow(windowDuration: Duration): DStream[(K, Iterable[V])] = { groupByKeyAndWindow(windowDuration, self.slideDuration, defaultPartitioner()) } @@ -140,7 +140,8 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * the new DStream will generate RDDs); must be a multiple of this * DStream's batching interval */ - def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration): DStream[(K, Seq[V])] = + def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration) + : DStream[(K, Iterable[V])] = { groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner()) } @@ -161,7 +162,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) windowDuration: Duration, slideDuration: Duration, numPartitions: Int - ): DStream[(K, Seq[V])] = { + ): DStream[(K, Iterable[V])] = { groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner(numPartitions)) } @@ -180,14 +181,14 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) windowDuration: Duration, slideDuration: Duration, partitioner: Partitioner - ): DStream[(K, Seq[V])] = { - val createCombiner = (v: Seq[V]) => new ArrayBuffer[V] ++= v - val mergeValue = (buf: ArrayBuffer[V], v: Seq[V]) => buf ++= v + ): DStream[(K, Iterable[V])] = { + val createCombiner = (v: Iterable[V]) => new ArrayBuffer[V] ++= v + val mergeValue = (buf: ArrayBuffer[V], v: Iterable[V]) => buf ++= v val mergeCombiner = (buf1: ArrayBuffer[V], buf2: ArrayBuffer[V]) => buf1 ++= buf2 self.groupByKey(partitioner) .window(windowDuration, slideDuration) .combineByKey[ArrayBuffer[V]](createCombiner, mergeValue, mergeCombiner, partitioner) - .asInstanceOf[DStream[(K, Seq[V])]] + .asInstanceOf[DStream[(K, Iterable[V])]] } /** @@ -438,7 +439,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Hash partitioning is used to generate the RDDs with Spark's default number * of partitions. */ - def cogroup[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = { + def cogroup[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner()) } @@ -447,7 +448,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. */ def cogroup[W: ClassTag](other: DStream[(K, W)], numPartitions: Int) - : DStream[(K, (Seq[V], Seq[W]))] = { + : DStream[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner(numPartitions)) } @@ -458,7 +459,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) def cogroup[W: ClassTag]( other: DStream[(K, W)], partitioner: Partitioner - ): DStream[(K, (Seq[V], Seq[W]))] = { + ): DStream[(K, (Iterable[V], Iterable[W]))] = { self.transformWith( other, (rdd1: RDD[(K, V)], rdd2: RDD[(K, W)]) => rdd1.cogroup(rdd2, partitioner) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala index 5f7d3ba26c656..7e22268767de7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala @@ -56,9 +56,14 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( // first map the cogrouped tuple to tuples of required type, // and then apply the update function val updateFuncLocal = updateFunc - val finalFunc = (iterator: Iterator[(K, (Seq[V], Seq[S]))]) => { + val finalFunc = (iterator: Iterator[(K, (Iterable[V], Iterable[S]))]) => { val i = iterator.map(t => { - (t._1, t._2._1, t._2._2.headOption) + val itr = t._2._2.iterator + val headOption = itr.hasNext match { + case true => Some(itr.next()) + case false => None + } + (t._1, t._2._1.toSeq, headOption) }) updateFuncLocal(i) } @@ -90,8 +95,8 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( // first map the grouped tuple to tuples of required type, // and then apply the update function val updateFuncLocal = updateFunc - val finalFunc = (iterator: Iterator[(K, Seq[V])]) => { - updateFuncLocal(iterator.map(tuple => (tuple._1, tuple._2, None))) + val finalFunc = (iterator: Iterator[(K, Iterable[V])]) => { + updateFuncLocal(iterator.map(tuple => (tuple._1, tuple._2.toSeq, None))) } val groupedRDD = parentRDD.groupByKey(partitioner) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index e93bf18b6d0b9..13fa64894b773 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -23,6 +23,7 @@ import org.junit.Test; import java.io.*; import java.util.*; +import java.lang.Iterable; import com.google.common.base.Optional; import com.google.common.collect.Lists; @@ -45,6 +46,18 @@ // see http://stackoverflow.com/questions/758570/. public class JavaAPISuite extends LocalJavaStreamingContext implements Serializable { + public void equalIterator(Iterator a, Iterator b) { + while (a.hasNext() && b.hasNext()) { + Assert.assertEquals(a.next(), b.next()); + } + Assert.assertEquals(a.hasNext(), b.hasNext()); + } + + public void equalIterable(Iterable a, Iterable b) { + equalIterator(a.iterator(), b.iterator()); + } + + @SuppressWarnings("unchecked") @Test public void testCount() { @@ -1016,11 +1029,24 @@ public void testPairGroupByKey() { JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream> grouped = pairStream.groupByKey(); + JavaPairDStream> grouped = pairStream.groupByKey(); JavaTestUtils.attachTestOutputStream(grouped); - List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); + List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected.size(), result.size()); + Iterator>>> resultItr = result.iterator(); + Iterator>>> expectedItr = expected.iterator(); + while (resultItr.hasNext() && expectedItr.hasNext()) { + Iterator>> resultElements = resultItr.next().iterator(); + Iterator>> expectedElements = expectedItr.next().iterator(); + while (resultElements.hasNext() && expectedElements.hasNext()) { + Tuple2> resultElement = resultElements.next(); + Tuple2> expectedElement = expectedElements.next(); + Assert.assertEquals(expectedElement._1(), resultElement._1()); + equalIterable(expectedElement._2(), resultElement._2()); + } + Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext()); + } } @SuppressWarnings("unchecked") @@ -1128,7 +1154,7 @@ public void testGroupByKeyAndWindow() { JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream> groupWindowed = + JavaPairDStream> groupWindowed = pairStream.groupByKeyAndWindow(new Duration(2000), new Duration(1000)); JavaTestUtils.attachTestOutputStream(groupWindowed); List>>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -1471,11 +1497,25 @@ public void testCoGroup() { ssc, stringStringKVStream2, 1); JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); - JavaPairDStream, List>> grouped = pairStream1.cogroup(pairStream2); + JavaPairDStream, Iterable>> grouped = pairStream1.cogroup(pairStream2); JavaTestUtils.attachTestOutputStream(grouped); - List, List>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); + List, Iterable>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected.size(), result.size()); + Iterator, Iterable>>>> resultItr = result.iterator(); + Iterator, List>>>> expectedItr = expected.iterator(); + while (resultItr.hasNext() && expectedItr.hasNext()) { + Iterator, Iterable>>> resultElements = resultItr.next().iterator(); + Iterator, List>>> expectedElements = expectedItr.next().iterator(); + while (resultElements.hasNext() && expectedElements.hasNext()) { + Tuple2, Iterable>> resultElement = resultElements.next(); + Tuple2, List>> expectedElement = expectedElements.next(); + Assert.assertEquals(expectedElement._1(), resultElement._1()); + equalIterable(expectedElement._2()._1(), resultElement._2()._1()); + equalIterable(expectedElement._2()._2(), resultElement._2()._2()); + } + Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext()); + } } @SuppressWarnings("unchecked") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index bb73dbf29b649..8aec27e39478a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -117,7 +117,7 @@ class BasicOperationsSuite extends TestSuiteBase { test("groupByKey") { testOperation( Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ), - (s: DStream[String]) => s.map(x => (x, 1)).groupByKey(), + (s: DStream[String]) => s.map(x => (x, 1)).groupByKey().mapValues(_.toSeq), Seq( Seq(("a", Seq(1, 1)), ("b", Seq(1))), Seq(("", Seq(1, 1))), Seq() ), true ) @@ -251,7 +251,7 @@ class BasicOperationsSuite extends TestSuiteBase { Seq( ) ) val operation = (s1: DStream[String], s2: DStream[String]) => { - s1.map(x => (x,1)).cogroup(s2.map(x => (x, "x"))) + s1.map(x => (x,1)).cogroup(s2.map(x => (x, "x"))).mapValues(x => (x._1.toSeq, x._2.toSeq)) } testOperation(inputData1, inputData2, operation, outputData, true) } From b9e0c937dfa1ca93b63d0b39d5f156b16c2fdc0a Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 8 Apr 2014 20:37:01 -0700 Subject: [PATCH 033/641] [SPARK-1434] [MLLIB] change labelParser from anonymous function to trait This is a patch to address @mateiz 's comment in https://github.com/apache/spark/pull/245 MLUtils#loadLibSVMData uses an anonymous function for the label parser. Java users won't like it. So I make a trait for LabelParser and provide two implementations: binary and multiclass. Author: Xiangrui Meng Closes #345 from mengxr/label-parser and squashes the following commits: ac44409 [Xiangrui Meng] use singleton objects for label parsers 3b1a7c6 [Xiangrui Meng] add tests for label parsers c2e571c [Xiangrui Meng] rename LabelParser.apply to LabelParser.parse use extends for singleton 11c94e0 [Xiangrui Meng] add return types 7f8eb36 [Xiangrui Meng] change labelParser from annoymous function to trait --- .../spark/mllib/util/LabelParsers.scala | 49 +++++++++++++++++++ .../org/apache/spark/mllib/util/MLUtils.scala | 28 ++--------- .../spark/mllib/util/LabelParsersSuite.scala | 41 ++++++++++++++++ .../spark/mllib/util/MLUtilsSuite.scala | 4 +- 4 files changed, 97 insertions(+), 25 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala new file mode 100644 index 0000000000000..f7966d3ebb613 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.util + +/** Trait for label parsers. */ +trait LabelParser extends Serializable { + /** Parses a string label into a double label. */ + def parse(labelString: String): Double +} + +/** + * Label parser for binary labels, which outputs 1.0 (positive) if the value is greater than 0.5, + * or 0.0 (negative) otherwise. So it works with +1/-1 labeling and +1/0 labeling. + */ +object BinaryLabelParser extends LabelParser { + /** Gets the default instance of BinaryLabelParser. */ + def getInstance(): LabelParser = this + + /** + * Parses the input label into positive (1.0) if the value is greater than 0.5, + * or negative (0.0) otherwise. + */ + override def parse(labelString: String): Double = if (labelString.toDouble > 0.5) 1.0 else 0.0 +} + +/** + * Label parser for multiclass labels, which converts the input label to double. + */ +object MulticlassLabelParser extends LabelParser { + /** Gets the default instance of MulticlassLabelParser. */ + def getInstance(): LabelParser = this + + override def parse(labelString: String): Double = labelString.toDouble +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index cb85e433bfc73..83d1bd3fd57fe 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -38,17 +38,6 @@ object MLUtils { eps } - /** - * Multiclass label parser, which parses a string into double. - */ - val multiclassLabelParser: String => Double = _.toDouble - - /** - * Binary label parser, which outputs 1.0 (positive) if the value is greater than 0.5, - * or 0.0 (negative) otherwise. - */ - val binaryLabelParser: String => Double = label => if (label.toDouble > 0.5) 1.0 else 0.0 - /** * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint]. * The LIBSVM format is a text-based format used by LIBSVM and LIBLINEAR. @@ -69,7 +58,7 @@ object MLUtils { def loadLibSVMData( sc: SparkContext, path: String, - labelParser: String => Double, + labelParser: LabelParser, numFeatures: Int, minSplits: Int): RDD[LabeledPoint] = { val parsed = sc.textFile(path, minSplits) @@ -89,7 +78,7 @@ object MLUtils { }.reduce(math.max) } parsed.map { items => - val label = labelParser(items.head) + val label = labelParser.parse(items.head) val (indices, values) = items.tail.map { item => val indexAndValue = item.split(':') val index = indexAndValue(0).toInt - 1 @@ -107,14 +96,7 @@ object MLUtils { * with number of features determined automatically and the default number of partitions. */ def loadLibSVMData(sc: SparkContext, path: String): RDD[LabeledPoint] = - loadLibSVMData(sc, path, binaryLabelParser, -1, sc.defaultMinSplits) - - /** - * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint], - * with number of features specified explicitly and the default number of partitions. - */ - def loadLibSVMData(sc: SparkContext, path: String, numFeatures: Int): RDD[LabeledPoint] = - loadLibSVMData(sc, path, binaryLabelParser, numFeatures, sc.defaultMinSplits) + loadLibSVMData(sc, path, BinaryLabelParser, -1, sc.defaultMinSplits) /** * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], @@ -124,7 +106,7 @@ object MLUtils { def loadLibSVMData( sc: SparkContext, path: String, - labelParser: String => Double): RDD[LabeledPoint] = + labelParser: LabelParser): RDD[LabeledPoint] = loadLibSVMData(sc, path, labelParser, -1, sc.defaultMinSplits) /** @@ -135,7 +117,7 @@ object MLUtils { def loadLibSVMData( sc: SparkContext, path: String, - labelParser: String => Double, + labelParser: LabelParser, numFeatures: Int): RDD[LabeledPoint] = loadLibSVMData(sc, path, labelParser, numFeatures, sc.defaultMinSplits) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala new file mode 100644 index 0000000000000..ac85677f2f014 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.util + +import org.scalatest.FunSuite + +class LabelParsersSuite extends FunSuite { + test("binary label parser") { + for (parser <- Seq(BinaryLabelParser, BinaryLabelParser.getInstance())) { + assert(parser.parse("+1") === 1.0) + assert(parser.parse("1") === 1.0) + assert(parser.parse("0") === 0.0) + assert(parser.parse("-1") === 0.0) + } + } + + test("multiclass label parser") { + for (parser <- Seq(MulticlassLabelParser, MulticlassLabelParser.getInstance())) { + assert(parser.parse("0") == 0.0) + assert(parser.parse("+1") === 1.0) + assert(parser.parse("1") === 1.0) + assert(parser.parse("2") === 2.0) + assert(parser.parse("3") === 3.0) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 27d41c7869aa0..e451c350b8d88 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -80,7 +80,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { Files.write(lines, file, Charsets.US_ASCII) val path = tempDir.toURI.toString - val pointsWithNumFeatures = MLUtils.loadLibSVMData(sc, path, 6).collect() + val pointsWithNumFeatures = MLUtils.loadLibSVMData(sc, path, BinaryLabelParser, 6).collect() val pointsWithoutNumFeatures = MLUtils.loadLibSVMData(sc, path).collect() for (points <- Seq(pointsWithNumFeatures, pointsWithoutNumFeatures)) { @@ -93,7 +93,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { assert(points(2).features === Vectors.sparse(6, Seq((1, 4.0), (3, 5.0), (5, 6.0)))) } - val multiclassPoints = MLUtils.loadLibSVMData(sc, path, MLUtils.multiclassLabelParser).collect() + val multiclassPoints = MLUtils.loadLibSVMData(sc, path, MulticlassLabelParser).collect() assert(multiclassPoints.length === 3) assert(multiclassPoints(0).label === 1.0) assert(multiclassPoints(1).label === -1.0) From fa0524fd02eedd0bbf1edc750dc3997a86ea25f5 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 8 Apr 2014 22:29:21 -0700 Subject: [PATCH 034/641] Spark-939: allow user jars to take precedence over spark jars I still need to do a small bit of re-factoring [mostly the one Java file I'll switch it back to a Scala file and use it in both the close loaders], but comments on other things I should do would be great. Author: Holden Karau Closes #217 from holdenk/spark-939-allow-user-jars-to-take-precedence-over-spark-jars and squashes the following commits: cf0cac9 [Holden Karau] Fix the executorclassloader 1955232 [Holden Karau] Fix long line in TestUtils 8f89965 [Holden Karau] Fix tests for new class name 7546549 [Holden Karau] CR feedback, merge some of the testutils methods down, rename the classloader 644719f [Holden Karau] User the class generator for the repl class loader tests too f0b7114 [Holden Karau] Fix the core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala tests 204b199 [Holden Karau] Fix the generated classes 9f68f10 [Holden Karau] Start rewriting the ExecutorURLClassLoaderSuite to not use the hard coded classes 858aba2 [Holden Karau] Remove a bunch of test junk 261aaee [Holden Karau] simplify executorurlclassloader a bit 7a7bf5f [Holden Karau] CR feedback d4ae848 [Holden Karau] rewrite component into scala aa95083 [Holden Karau] CR feedback 7752594 [Holden Karau] re-add https comment a0ef85a [Holden Karau] Fix style issues 125ea7f [Holden Karau] Easier to just remove those files, we don't need them bb8d179 [Holden Karau] Fix issues with the repl class loader 241b03d [Holden Karau] fix my rat excludes a343350 [Holden Karau] Update rat-excludes and remove a useless file d90d217 [Holden Karau] Fix fall back with custom class loader and add a test for it 4919bf9 [Holden Karau] Fix parent calling class loader issue 8a67302 [Holden Karau] Test are good 9e2d236 [Holden Karau] It works comrade 691ee00 [Holden Karau] It works ish dc4fe44 [Holden Karau] Does not depend on being in my home directory 47046ff [Holden Karau] Remove bad import' 22d83cb [Holden Karau] Add a test suite for the executor url class loader suite 7ef4628 [Holden Karau] Clean up 792d961 [Holden Karau] Almost works 16aecd1 [Holden Karau] Doesn't quite work 8d2241e [Holden Karau] Adda FakeClass for testing ClassLoader precedence options 648b559 [Holden Karau] Both class loaders compile. Now for testing e1d9f71 [Holden Karau] One loader workers. --- .rat-excludes | 2 +- .../scala/org/apache/spark/TestUtils.scala | 20 +++-- .../org/apache/spark/executor/Executor.scala | 17 +++-- .../executor/ExecutorURLClassLoader.scala | 45 ++++++++++- .../apache/spark/util/ParentClassLoader.scala | 32 ++++++++ .../ExecutorURLClassLoaderSuite.scala | 67 ++++++++++++++++ docs/configuration.md | 9 +++ project/SparkBuild.scala | 1 + .../spark/repl/ExecutorClassLoader.scala | 39 +++++++--- .../spark/repl/ExecutorClassLoaderSuite.scala | 76 +++++++++++++++++++ 10 files changed, 287 insertions(+), 21 deletions(-) rename core/src/{test => main}/scala/org/apache/spark/TestUtils.scala (84%) create mode 100644 core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala create mode 100644 core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala create mode 100644 repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala diff --git a/.rat-excludes b/.rat-excludes index 85bfad60fcadc..a2b5665a0be26 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -39,4 +39,4 @@ work .*\.q golden test.out/* -.*iml +.*iml \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala similarity index 84% rename from core/src/test/scala/org/apache/spark/TestUtils.scala rename to core/src/main/scala/org/apache/spark/TestUtils.scala index 1611d09652d40..4597595a838e3 100644 --- a/core/src/test/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -26,7 +26,14 @@ import scala.collection.JavaConversions._ import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} import com.google.common.io.Files -object TestUtils { +/** + * Utilities for tests. Included in main codebase since it's used by multiple + * projects. + * + * TODO: See if we can move this to the test codebase by specifying + * test dependencies between projects. + */ +private[spark] object TestUtils { /** * Create a jar that defines classes with the given names. @@ -34,13 +41,14 @@ object TestUtils { * Note: if this is used during class loader tests, class names should be unique * in order to avoid interference between tests. */ - def createJarWithClasses(classNames: Seq[String]): URL = { + def createJarWithClasses(classNames: Seq[String], value: String = ""): URL = { val tempDir = Files.createTempDir() - val files = for (name <- classNames) yield createCompiledClass(name, tempDir) + val files = for (name <- classNames) yield createCompiledClass(name, tempDir, value) val jarFile = new File(tempDir, "testJar-%s.jar".format(System.currentTimeMillis())) createJar(files, jarFile) } + /** * Create a jar file that contains this set of files. All files will be located at the root * of the jar. @@ -80,9 +88,11 @@ object TestUtils { } /** Creates a compiled class with the given name. Class file will be placed in destDir. */ - def createCompiledClass(className: String, destDir: File): File = { + def createCompiledClass(className: String, destDir: File, value: String = ""): File = { val compiler = ToolProvider.getSystemJavaCompiler - val sourceFile = new JavaSourceFromString(className, s"public class $className {}") + val sourceFile = new JavaSourceFromString(className, + "public class " + className + " { @Override public String toString() { " + + "return \"" + value + "\";}}") // Calling this outputs a class file in pwd. It's easier to just rename the file than // build a custom FileManager that controls the output location. 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 aecb069e4202b..c12bd922d40e4 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -291,15 +291,19 @@ private[spark] class Executor( * Create a ClassLoader for use in tasks, adding any JARs specified by the user or any classes * created by the interpreter to the search path */ - private def createClassLoader(): ExecutorURLClassLoader = { - val loader = Thread.currentThread().getContextClassLoader + private def createClassLoader(): MutableURLClassLoader = { + val loader = this.getClass.getClassLoader // For each of the jars in the jarSet, add them to the class loader. // We assume each of the files has already been fetched. val urls = currentJars.keySet.map { uri => new File(uri.split("/").last).toURI.toURL }.toArray - new ExecutorURLClassLoader(urls, loader) + val userClassPathFirst = conf.getBoolean("spark.files.userClassPathFirst", false) + userClassPathFirst match { + case true => new ChildExecutorURLClassLoader(urls, loader) + case false => new ExecutorURLClassLoader(urls, loader) + } } /** @@ -310,11 +314,14 @@ private[spark] class Executor( val classUri = conf.get("spark.repl.class.uri", null) if (classUri != null) { logInfo("Using REPL class URI: " + classUri) + val userClassPathFirst: java.lang.Boolean = + conf.getBoolean("spark.files.userClassPathFirst", false) try { val klass = Class.forName("org.apache.spark.repl.ExecutorClassLoader") .asInstanceOf[Class[_ <: ClassLoader]] - val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader]) - constructor.newInstance(classUri, parent) + val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader], + classOf[Boolean]) + constructor.newInstance(classUri, parent, userClassPathFirst) } catch { case _: ClassNotFoundException => logError("Could not find org.apache.spark.repl.ExecutorClassLoader on classpath!") diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala index f9bfe8ed2f5ba..208e77073fd03 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala @@ -19,13 +19,56 @@ package org.apache.spark.executor import java.net.{URLClassLoader, URL} +import org.apache.spark.util.ParentClassLoader + /** * The addURL method in URLClassLoader is protected. We subclass it to make this accessible. + * We also make changes so user classes can come before the default classes. */ + +private[spark] trait MutableURLClassLoader extends ClassLoader { + def addURL(url: URL) + def getURLs: Array[URL] +} + +private[spark] class ChildExecutorURLClassLoader(urls: Array[URL], parent: ClassLoader) + extends MutableURLClassLoader { + + private object userClassLoader extends URLClassLoader(urls, null){ + override def addURL(url: URL) { + super.addURL(url) + } + override def findClass(name: String): Class[_] = { + super.findClass(name) + } + } + + private val parentClassLoader = new ParentClassLoader(parent) + + override def findClass(name: String): Class[_] = { + try { + userClassLoader.findClass(name) + } catch { + case e: ClassNotFoundException => { + parentClassLoader.loadClass(name) + } + } + } + + def addURL(url: URL) { + userClassLoader.addURL(url) + } + + def getURLs() = { + userClassLoader.getURLs() + } +} + private[spark] class ExecutorURLClassLoader(urls: Array[URL], parent: ClassLoader) - extends URLClassLoader(urls, parent) { + extends URLClassLoader(urls, parent) with MutableURLClassLoader { override def addURL(url: URL) { super.addURL(url) } } + diff --git a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala new file mode 100644 index 0000000000000..3abc12681fe9a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +/** + * A class loader which makes findClass accesible to the child + */ +private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader(parent) { + + override def findClass(name: String) = { + super.findClass(name) + } + + override def loadClass(name: String): Class[_] = { + super.loadClass(name) + } +} diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala new file mode 100644 index 0000000000000..c40cfc0696fce --- /dev/null +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import java.io.File +import java.net.URLClassLoader + +import org.scalatest.FunSuite + +import org.apache.spark.TestUtils + +class ExecutorURLClassLoaderSuite extends FunSuite { + + val childClassNames = List("FakeClass1", "FakeClass2") + val parentClassNames = List("FakeClass1", "FakeClass2", "FakeClass3") + val urls = List(TestUtils.createJarWithClasses(childClassNames, "1")).toArray + val urls2 = List(TestUtils.createJarWithClasses(parentClassNames, "2")).toArray + + test("child first") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + val fakeClass = classLoader.loadClass("FakeClass2").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "1") + } + + test("parent first") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ExecutorURLClassLoader(urls, parentLoader) + val fakeClass = classLoader.loadClass("FakeClass1").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "2") + } + + test("child first can fall back") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + val fakeClass = classLoader.loadClass("FakeClass3").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "2") + } + + test("child first can fail") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + intercept[java.lang.ClassNotFoundException] { + classLoader.loadClass("FakeClassDoesNotExist").newInstance() + } + } + + +} diff --git a/docs/configuration.md b/docs/configuration.md index 57bda20edcdf1..9c602402f0635 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -596,6 +596,15 @@ Apart from these, the following properties are also available, and may be useful the driver. + + spark.files.userClassPathFirst + false + + (Experimental) Whether to give user-added jars precedence over Spark's own jars when + loading classes in Executors. This feature can be used to mitigate conflicts between + Spark's dependencies and user dependencies. It is currently an experimental feature. + + spark.authenticate false diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 08667aac2cd2d..694f90a83ab67 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -195,6 +195,7 @@ object SparkBuild extends Build { concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), resolvers ++= Seq( + // HTTPS is unavailable for Maven Central "Maven Repository" at "http://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/", diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index bf73800388ebf..a30dcfdcecf27 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -26,21 +26,23 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.SparkEnv import org.apache.spark.util.Utils - +import org.apache.spark.util.ParentClassLoader import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm._ import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ - /** * A ClassLoader that reads classes from a Hadoop FileSystem or HTTP URI, - * used to load classes defined by the interpreter when the REPL is used - */ -class ExecutorClassLoader(classUri: String, parent: ClassLoader) -extends ClassLoader(parent) { + * used to load classes defined by the interpreter when the REPL is used. + * Allows the user to specify if user class path should be first + */ +class ExecutorClassLoader(classUri: String, parent: ClassLoader, + userClassPathFirst: Boolean) extends ClassLoader { val uri = new URI(classUri) val directory = uri.getPath + val parentLoader = new ParentClassLoader(parent) + // Hadoop FileSystem object for our URI, if it isn't using HTTP var fileSystem: FileSystem = { if (uri.getScheme() == "http") { @@ -49,8 +51,27 @@ extends ClassLoader(parent) { FileSystem.get(uri, new Configuration()) } } - + override def findClass(name: String): Class[_] = { + userClassPathFirst match { + case true => findClassLocally(name).getOrElse(parentLoader.loadClass(name)) + case false => { + try { + parentLoader.loadClass(name) + } catch { + case e: ClassNotFoundException => { + val classOption = findClassLocally(name) + classOption match { + case None => throw new ClassNotFoundException(name, e) + case Some(a) => a + } + } + } + } + } + } + + def findClassLocally(name: String): Option[Class[_]] = { try { val pathInDirectory = name.replace('.', '/') + ".class" val inputStream = { @@ -68,9 +89,9 @@ extends ClassLoader(parent) { } val bytes = readAndTransformClass(name, inputStream) inputStream.close() - return defineClass(name, bytes, 0, bytes.length) + Some(defineClass(name, bytes, 0, bytes.length)) } catch { - case e: Exception => throw new ClassNotFoundException(name, e) + case e: Exception => None } } diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala new file mode 100644 index 0000000000000..336df988a1b7f --- /dev/null +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -0,0 +1,76 @@ +/* + * 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.repl + +import java.io.File +import java.net.URLClassLoader + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import com.google.common.io.Files + +import org.apache.spark.TestUtils + +class ExecutorClassLoaderSuite extends FunSuite with BeforeAndAfterAll { + + val childClassNames = List("ReplFakeClass1", "ReplFakeClass2") + val parentClassNames = List("ReplFakeClass1", "ReplFakeClass2", "ReplFakeClass3") + val tempDir1 = Files.createTempDir() + val tempDir2 = Files.createTempDir() + val url1 = "file://" + tempDir1 + val urls2 = List(tempDir2.toURI.toURL).toArray + + override def beforeAll() { + childClassNames.foreach(TestUtils.createCompiledClass(_, tempDir1, "1")) + parentClassNames.foreach(TestUtils.createCompiledClass(_, tempDir2, "2")) + } + + test("child first") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ExecutorClassLoader(url1, parentLoader, true) + val fakeClass = classLoader.loadClass("ReplFakeClass2").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "1") + } + + test("parent first") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ExecutorClassLoader(url1, parentLoader, false) + val fakeClass = classLoader.loadClass("ReplFakeClass1").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "2") + } + + test("child first can fall back") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ExecutorClassLoader(url1, parentLoader, true) + val fakeClass = classLoader.loadClass("ReplFakeClass3").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "2") + } + + test("child first can fail") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ExecutorClassLoader(url1, parentLoader, true) + intercept[java.lang.ClassNotFoundException] { + classLoader.loadClass("ReplFakeClassDoesNotExist").newInstance() + } + } + +} From 9689b663a2a4947ad60795321c770052f3c637f1 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 8 Apr 2014 23:01:15 -0700 Subject: [PATCH 035/641] [SPARK-1390] Refactoring of matrices backed by RDDs This is to refactor interfaces for matrices backed by RDDs. It would be better if we have a clear separation of local matrices and those backed by RDDs. Right now, we have 1. `org.apache.spark.mllib.linalg.SparseMatrix`, which is a wrapper over an RDD of matrix entries, i.e., coordinate list format. 2. `org.apache.spark.mllib.linalg.TallSkinnyDenseMatrix`, which is a wrapper over RDD[Array[Double]], i.e. row-oriented format. We will see naming collision when we introduce local `SparseMatrix`, and the name `TallSkinnyDenseMatrix` is not exact if we switch to `RDD[Vector]` from `RDD[Array[Double]]`. It would be better to have "RDD" in the class name to suggest that operations may trigger jobs. The proposed names are (all under `org.apache.spark.mllib.linalg.rdd`): 1. `RDDMatrix`: trait for matrices backed by one or more RDDs 2. `CoordinateRDDMatrix`: wrapper of `RDD[(Long, Long, Double)]` 3. `RowRDDMatrix`: wrapper of `RDD[Vector]` whose rows do not have special ordering 4. `IndexedRowRDDMatrix`: wrapper of `RDD[(Long, Vector)]` whose rows are associated with indices The current code also introduces local matrices. Author: Xiangrui Meng Closes #296 from mengxr/mat and squashes the following commits: 24d8294 [Xiangrui Meng] fix for groupBy returning Iterable bfc2b26 [Xiangrui Meng] merge master 8e4f1f5 [Xiangrui Meng] Merge branch 'master' into mat 0135193 [Xiangrui Meng] address Reza's comments 03cd7e1 [Xiangrui Meng] add pca/gram to IndexedRowMatrix add toBreeze to DistributedMatrix for test simplify tests b177ff1 [Xiangrui Meng] address Matei's comments be119fe [Xiangrui Meng] rename m/n to numRows/numCols for local matrix add tests for matrices b881506 [Xiangrui Meng] rename SparkPCA/SVD to TallSkinnyPCA/SVD e7d0d4a [Xiangrui Meng] move IndexedRDDMatrixRow to IndexedRowRDDMatrix 0d1491c [Xiangrui Meng] fix test errors a85262a [Xiangrui Meng] rename RDDMatrixRow to IndexedRDDMatrixRow b8b6ac3 [Xiangrui Meng] Remove old code 4cf679c [Xiangrui Meng] port pca to RowRDDMatrix, and add multiply and covariance 7836e2f [Xiangrui Meng] initial refactoring of matrices backed by RDDs --- .../spark/examples/mllib/SparkPCA.scala | 51 --- .../spark/examples/mllib/SparkSVD.scala | 59 --- .../spark/examples/mllib/TallSkinnyPCA.scala | 64 +++ .../spark/examples/mllib/TallSkinnySVD.scala | 64 +++ .../apache/spark/mllib/linalg/Matrices.scala | 101 +++++ .../apache/spark/mllib/linalg/MatrixSVD.scala | 29 -- .../org/apache/spark/mllib/linalg/PCA.scala | 120 ------ .../org/apache/spark/mllib/linalg/SVD.scala | 395 ------------------ ...scala => SingularValueDecomposition.scala} | 9 +- .../mllib/linalg/TallSkinnyMatrixSVD.scala | 31 -- .../linalg/distributed/CoordinateMatrix.scala | 112 +++++ .../DistributedMatrix.scala} | 23 +- .../linalg/distributed/IndexedRowMatrix.scala | 148 +++++++ .../mllib/linalg/distributed/RowMatrix.scala | 344 +++++++++++++++ .../org/apache/spark/mllib/util/LAUtils.scala | 67 --- .../linalg/BreezeMatrixConversionSuite.scala} | 29 +- .../spark/mllib/linalg/MatricesSuite.scala} | 27 +- .../apache/spark/mllib/linalg/PCASuite.scala | 124 ------ .../apache/spark/mllib/linalg/SVDSuite.scala | 194 --------- .../distributed/CoordinateMatrixSuite.scala | 98 +++++ .../distributed/IndexedRowMatrixSuite.scala | 120 ++++++ .../linalg/distributed/RowMatrixSuite.scala | 173 ++++++++ 22 files changed, 1280 insertions(+), 1102 deletions(-) delete mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala delete mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixSVD.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala rename mllib/src/main/scala/org/apache/spark/mllib/linalg/{MatrixRow.scala => SingularValueDecomposition.scala} (81%) delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala rename mllib/src/main/scala/org/apache/spark/mllib/linalg/{SparseMatrix.scala => distributed/DistributedMatrix.scala} (60%) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala rename mllib/src/{main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala => test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala} (51%) rename mllib/src/{main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala => test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala} (58%) delete mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala delete mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala deleted file mode 100644 index d4e08c5e12d81..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala +++ /dev/null @@ -1,51 +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.examples.mllib - -import org.apache.spark.SparkContext -import org.apache.spark.mllib.linalg.PCA -import org.apache.spark.mllib.linalg.MatrixEntry -import org.apache.spark.mllib.linalg.SparseMatrix -import org.apache.spark.mllib.util._ - - -/** - * Compute PCA of an example matrix. - */ -object SparkPCA { - def main(args: Array[String]) { - if (args.length != 3) { - System.err.println("Usage: SparkPCA m n") - System.exit(1) - } - val sc = new SparkContext(args(0), "PCA", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) - - val m = args(2).toInt - val n = args(3).toInt - - // Make example matrix - val data = Array.tabulate(m, n) { (a, b) => - (a + 2).toDouble * (b + 1) / (1 + a + b) } - - // recover top principal component - val coeffs = new PCA().setK(1).compute(sc.makeRDD(data)) - - println("top principal component = " + coeffs.mkString(", ")) - } -} 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 deleted file mode 100644 index 2933cec497b37..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala +++ /dev/null @@ -1,59 +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.examples.mllib - -import org.apache.spark.SparkContext -import org.apache.spark.mllib.linalg.SVD -import org.apache.spark.mllib.linalg.MatrixEntry -import org.apache.spark.mllib.linalg.SparseMatrix - -/** - * Compute SVD of an example matrix - * Input file should be comma separated, 1 indexed of the form - * i,j,value - * Where i is the column, j the row, and value is the matrix entry - * - * For example input file, see: - * mllib/data/als/test.data (example is 4 x 4) - */ -object SparkSVD { - def main(args: Array[String]) { - if (args.length != 4) { - System.err.println("Usage: SparkSVD m n") - System.exit(1) - } - val sc = new SparkContext(args(0), "SVD", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) - - // Load and parse the data file - val data = sc.textFile(args(1)).map { line => - val parts = line.split(',') - MatrixEntry(parts(0).toInt - 1, parts(1).toInt - 1, parts(2).toDouble) - } - val m = args(2).toInt - val n = args(3).toInt - - // recover largest singular vector - val decomposed = new SVD().setK(1).compute(SparseMatrix(data, m, n)) - val u = decomposed.U.data - val s = decomposed.S.data - val v = decomposed.V.data - - println("singular values = " + s.collect().mkString) - } -} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala new file mode 100644 index 0000000000000..a177435e606ab --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -0,0 +1,64 @@ +/* + * 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.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.linalg.Vectors + +/** + * Compute the principal components of a tall-and-skinny matrix, whose rows are observations. + * + * The input matrix must be stored in row-oriented dense format, one line per row with its entries + * separated by space. For example, + * {{{ + * 0.5 1.0 + * 2.0 3.0 + * 4.0 5.0 + * }}} + * represents a 3-by-2 matrix, whose first row is (0.5, 1.0). + */ +object TallSkinnyPCA { + def main(args: Array[String]) { + if (args.length != 2) { + System.err.println("Usage: TallSkinnyPCA ") + System.exit(1) + } + + val conf = new SparkConf() + .setMaster(args(0)) + .setAppName("TallSkinnyPCA") + .setSparkHome(System.getenv("SPARK_HOME")) + .setJars(SparkContext.jarOfClass(this.getClass)) + val sc = new SparkContext(conf) + + // Load and parse the data file. + val rows = sc.textFile(args(1)).map { line => + val values = line.split(' ').map(_.toDouble) + Vectors.dense(values) + } + val mat = new RowMatrix(rows) + + // Compute principal components. + val pc = mat.computePrincipalComponents(mat.numCols().toInt) + + println("Principal components are:\n" + pc) + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala new file mode 100644 index 0000000000000..49d09692c8e4a --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -0,0 +1,64 @@ +/* + * 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.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.linalg.Vectors + +/** + * Compute the singular value decomposition (SVD) of a tall-and-skinny matrix. + * + * The input matrix must be stored in row-oriented dense format, one line per row with its entries + * separated by space. For example, + * {{{ + * 0.5 1.0 + * 2.0 3.0 + * 4.0 5.0 + * }}} + * represents a 3-by-2 matrix, whose first row is (0.5, 1.0). + */ +object TallSkinnySVD { + def main(args: Array[String]) { + if (args.length != 2) { + System.err.println("Usage: TallSkinnySVD ") + System.exit(1) + } + + val conf = new SparkConf() + .setMaster(args(0)) + .setAppName("TallSkinnySVD") + .setSparkHome(System.getenv("SPARK_HOME")) + .setJars(SparkContext.jarOfClass(this.getClass)) + val sc = new SparkContext(conf) + + // Load and parse the data file. + val rows = sc.textFile(args(1)).map { line => + val values = line.split(' ').map(_.toDouble) + Vectors.dense(values) + } + val mat = new RowMatrix(rows) + + // Compute SVD. + val svd = mat.computeSVD(mat.numCols().toInt) + + println("Singular values are " + svd.s) + + sc.stop() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala new file mode 100644 index 0000000000000..b11ba5d30fbd3 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg + +import breeze.linalg.{Matrix => BM, DenseMatrix => BDM} + +/** + * Trait for a local matrix. + */ +trait Matrix extends Serializable { + + /** Number of rows. */ + def numRows: Int + + /** Number of columns. */ + def numCols: Int + + /** Converts to a dense array in column major. */ + def toArray: Array[Double] + + /** Converts to a breeze matrix. */ + private[mllib] def toBreeze: BM[Double] + + /** Gets the (i, j)-th element. */ + private[mllib] def apply(i: Int, j: Int): Double = toBreeze(i, j) + + override def toString: String = toBreeze.toString() +} + +/** + * Column-majored dense matrix. + * The entry values are stored in a single array of doubles with columns listed in sequence. + * For example, the following matrix + * {{{ + * 1.0 2.0 + * 3.0 4.0 + * 5.0 6.0 + * }}} + * is stored as `[1.0, 3.0, 5.0, 2.0, 4.0, 6.0]`. + * + * @param numRows number of rows + * @param numCols number of columns + * @param values matrix entries in column major + */ +class DenseMatrix(val numRows: Int, val numCols: Int, val values: Array[Double]) extends Matrix { + + require(values.length == numRows * numCols) + + override def toArray: Array[Double] = values + + private[mllib] override def toBreeze: BM[Double] = new BDM[Double](numRows, numCols, values) +} + +/** + * Factory methods for [[org.apache.spark.mllib.linalg.Matrix]]. + */ +object Matrices { + + /** + * Creates a column-majored dense matrix. + * + * @param numRows number of rows + * @param numCols number of columns + * @param values matrix entries in column major + */ + def dense(numRows: Int, numCols: Int, values: Array[Double]): Matrix = { + new DenseMatrix(numRows, numCols, values) + } + + /** + * Creates a Matrix instance from a breeze matrix. + * @param breeze a breeze matrix + * @return a Matrix instance + */ + private[mllib] def fromBreeze(breeze: BM[Double]): Matrix = { + breeze match { + case dm: BDM[Double] => + require(dm.majorStride == dm.rows, + "Do not support stride size different from the number of rows.") + new DenseMatrix(dm.rows, dm.cols, dm.data) + case _ => + throw new UnsupportedOperationException( + s"Do not support conversion from type ${breeze.getClass.getName}.") + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixSVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixSVD.scala deleted file mode 100644 index 319f82b449096..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixSVD.scala +++ /dev/null @@ -1,29 +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.mllib.linalg - -/** - * Class that represents the SV decomposition of a matrix - * - * @param U such that A = USV^T - * @param S such that A = USV^T - * @param V such that A = USV^T - */ -case class MatrixSVD(val U: SparseMatrix, - val S: SparseMatrix, - val V: SparseMatrix) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala deleted file mode 100644 index fe5b3f6c7e463..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala +++ /dev/null @@ -1,120 +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.mllib.linalg - -import org.apache.spark.rdd.RDD - - -import org.jblas.DoubleMatrix - - -/** - * Class used to obtain principal components - */ -class PCA { - private var k = 1 - - /** - * Set the number of top-k principle components to return - */ - def setK(k: Int): PCA = { - this.k = k - this - } - - /** - * Compute PCA using the current set parameters - */ - def compute(matrix: TallSkinnyDenseMatrix): Array[Array[Double]] = { - computePCA(matrix) - } - - /** - * Compute PCA using the parameters currently set - * See computePCA() for more details - */ - def compute(matrix: RDD[Array[Double]]): Array[Array[Double]] = { - computePCA(matrix) - } - - /** - * Computes the top k principal component coefficients for the m-by-n data matrix X. - * Rows of X correspond to observations and columns correspond to variables. - * The coefficient matrix is n-by-k. Each column of coeff contains coefficients - * for one principal component, and the columns are in descending - * order of component variance. - * This function centers the data and uses the - * singular value decomposition (SVD) algorithm. - * - * @param matrix dense matrix to perform PCA on - * @return An nxk matrix with principal components in columns. Columns are inner arrays - */ - private def computePCA(matrix: TallSkinnyDenseMatrix): Array[Array[Double]] = { - val m = matrix.m - val n = matrix.n - - if (m <= 0 || n <= 0) { - throw new IllegalArgumentException("Expecting a well-formed matrix: m=$m n=$n") - } - - computePCA(matrix.rows.map(_.data)) - } - - /** - * Computes the top k principal component coefficients for the m-by-n data matrix X. - * Rows of X correspond to observations and columns correspond to variables. - * The coefficient matrix is n-by-k. Each column of coeff contains coefficients - * for one principal component, and the columns are in descending - * order of component variance. - * This function centers the data and uses the - * singular value decomposition (SVD) algorithm. - * - * @param matrix dense matrix to perform pca on - * @return An nxk matrix of principal components - */ - private def computePCA(matrix: RDD[Array[Double]]): Array[Array[Double]] = { - val n = matrix.first.size - - // compute column sums and normalize matrix - val colSumsTemp = matrix.map((_, 1)).fold((Array.ofDim[Double](n), 0)) { - (a, b) => - val am = new DoubleMatrix(a._1) - val bm = new DoubleMatrix(b._1) - am.addi(bm) - (a._1, a._2 + b._2) - } - - val m = colSumsTemp._2 - val colSums = colSumsTemp._1.map(x => x / m) - - val data = matrix.map { - x => - val row = Array.ofDim[Double](n) - var i = 0 - while (i < n) { - row(i) = x(i) - colSums(i) - i += 1 - } - row - } - - val (u, s, v) = new SVD().setK(k).compute(data) - v - } -} - 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 deleted file mode 100644 index 0d97b7d92f155..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala +++ /dev/null @@ -1,395 +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.mllib.linalg - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - -import org.jblas.{DoubleMatrix, Singular, MatrixFunctions} - -/** - * Class used to obtain singular value decompositions - */ -class SVD { - private var k = 1 - private var computeU = true - - // All singular values smaller than rCond * sigma(0) - // are treated as zero, where sigma(0) is the largest singular value. - private var rCond = 1e-9 - - /** - * Set the number of top-k singular vectors to return - */ - def setK(k: Int): SVD = { - this.k = k - this - } - - /** - * Sets the reciprocal condition number (rCond). All singular values - * smaller than rCond * sigma(0) are treated as zero, - * where sigma(0) is the largest singular value. - */ - def setReciprocalConditionNumber(smallS: Double): SVD = { - this.rCond = smallS - this - } - - /** - * Should U be computed? - */ - def setComputeU(compU: Boolean): SVD = { - this.computeU = compU - this - } - - /** - * Compute SVD using the current set parameters - */ - def compute(matrix: TallSkinnyDenseMatrix): TallSkinnyMatrixSVD = { - denseSVD(matrix) - } - - /** - * Compute SVD using the current set parameters - * Returns (U, S, V) such that A = USV^T - * U is a row-by-row dense matrix - * S is a simple double array of singular values - * V is a 2d array matrix - * See [[denseSVD]] for more documentation - */ - def compute(matrix: RDD[Array[Double]]): - (RDD[Array[Double]], Array[Double], Array[Array[Double]]) = { - denseSVD(matrix) - } - - /** - * See full paramter definition of sparseSVD for more description. - * - * @param matrix sparse matrix to factorize - * @return Three sparse matrices: U, S, V such that A = USV^T - */ - def compute(matrix: SparseMatrix): MatrixSVD = { - sparseSVD(matrix) - } - - /** - * Singular Value Decomposition for Tall and Skinny matrices. - * Given an m x n matrix A, this will compute matrices U, S, V such that - * A = U * S * V' - * - * There is no restriction on m, but we require n^2 doubles to fit in memory. - * Further, n should be less than m. - * - * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), - * from which we recover S and V. - * Then we compute U via easy matrix multiplication - * as U = A * V * S^-1 - * - * Only the k largest singular values and associated vectors are found. - * If there are k such values, then the dimensions of the return will be: - * - * S is k x k and diagonal, holding the singular values on diagonal - * U is m x k and satisfies U'U = eye(k) - * V is n x k and satisfies V'V = eye(k) - * - * @param matrix dense matrix to factorize - * @return See [[TallSkinnyMatrixSVD]] for the output matrices and arrays - */ - private def denseSVD(matrix: TallSkinnyDenseMatrix): TallSkinnyMatrixSVD = { - val m = matrix.m - val n = matrix.n - - if (m < n || m <= 0 || n <= 0) { - throw new IllegalArgumentException("Expecting a tall and skinny matrix m=$m n=$n") - } - - if (k < 1 || k > n) { - throw new IllegalArgumentException("Request up to n singular values n=$n k=$k") - } - - val rowIndices = matrix.rows.map(_.i) - - // compute SVD - val (u, sigma, v) = denseSVD(matrix.rows.map(_.data)) - - if (computeU) { - // prep u for returning - val retU = TallSkinnyDenseMatrix( - u.zip(rowIndices).map { - case (row, i) => MatrixRow(i, row) - }, - m, - k) - - TallSkinnyMatrixSVD(retU, sigma, v) - } else { - TallSkinnyMatrixSVD(null, sigma, v) - } - } - - /** - * Singular Value Decomposition for Tall and Skinny matrices. - * Given an m x n matrix A, this will compute matrices U, S, V such that - * A = U * S * V' - * - * There is no restriction on m, but we require n^2 doubles to fit in memory. - * Further, n should be less than m. - * - * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), - * from which we recover S and V. - * Then we compute U via easy matrix multiplication - * as U = A * V * S^-1 - * - * Only the k largest singular values and associated vectors are found. - * If there are k such values, then the dimensions of the return will be: - * - * S is k x k and diagonal, holding the singular values on diagonal - * U is m x k and satisfies U'U = eye(k) - * V is n x k and satisfies V'V = eye(k) - * - * The return values are as lean as possible: an RDD of rows for U, - * a simple array for sigma, and a dense 2d matrix array for V - * - * @param matrix dense matrix to factorize - * @return Three matrices: U, S, V such that A = USV^T - */ - private def denseSVD(matrix: RDD[Array[Double]]): - (RDD[Array[Double]], Array[Double], Array[Array[Double]]) = { - val n = matrix.first.size - - if (k < 1 || k > n) { - throw new IllegalArgumentException( - "Request up to n singular values k=$k n=$n") - } - - // Compute A^T A - val fullata = matrix.mapPartitions { - iter => - val localATA = Array.ofDim[Double](n, n) - while (iter.hasNext) { - val row = iter.next() - var i = 0 - while (i < n) { - var j = 0 - while (j < n) { - localATA(i)(j) += row(i) * row(j) - j += 1 - } - i += 1 - } - } - Iterator(localATA) - }.fold(Array.ofDim[Double](n, n)) { - (a, b) => - var i = 0 - while (i < n) { - var j = 0 - while (j < n) { - a(i)(j) += b(i)(j) - j += 1 - } - i += 1 - } - a - } - - // Construct jblas A^T A locally - val ata = new DoubleMatrix(fullata) - - // Since A^T A is small, we can compute its SVD directly - val svd = Singular.sparseSVD(ata) - val V = svd(0) - val sigmas = MatrixFunctions.sqrt(svd(1)).toArray.filter(x => x / svd(1).get(0) > rCond) - - val sk = Math.min(k, sigmas.size) - val sigma = sigmas.take(sk) - - // prepare V for returning - val retV = Array.tabulate(n, sk)((i, j) => V.get(i, j)) - - if (computeU) { - // Compute U as U = A V S^-1 - // Compute VS^-1 - val vsinv = new DoubleMatrix(Array.tabulate(n, sk)((i, j) => V.get(i, j) / sigma(j))) - val retU = matrix.map { - x => - val v = new DoubleMatrix(Array(x)) - v.mmul(vsinv).data - } - (retU, sigma, retV) - } else { - (null, sigma, retV) - } - } - - /** - * Singular Value Decomposition for Tall and Skinny sparse matrices. - * Given an m x n matrix A, this will compute matrices U, S, V such that - * A = U * S * V' - * - * There is no restriction on m, but we require O(n^2) doubles to fit in memory. - * Further, n should be less than m. - * - * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), - * from which we recover S and V. - * Then we compute U via easy matrix multiplication - * as U = A * V * S^-1 - * - * Only the k largest singular values and associated vectors are found. - * If there are k such values, then the dimensions of the return will be: - * - * S is k x k and diagonal, holding the singular values on diagonal - * U is m x k and satisfies U'U = eye(k) - * V is n x k and satisfies V'V = eye(k) - * - * All input and output is expected in sparse matrix format, 0-indexed - * as tuples of the form ((i,j),value) all in RDDs using the - * SparseMatrix class - * - * @param matrix sparse matrix to factorize - * @return Three sparse matrices: U, S, V such that A = USV^T - */ - private def sparseSVD(matrix: SparseMatrix): MatrixSVD = { - val data = matrix.data - val m = matrix.m - val n = matrix.n - - if (m < n || m <= 0 || n <= 0) { - throw new IllegalArgumentException("Expecting a tall and skinny matrix") - } - - if (k < 1 || k > n) { - throw new IllegalArgumentException("Must request up to n singular values") - } - - // Compute A^T A, assuming rows are sparse enough to fit in memory - val rows = data.map(entry => - (entry.i, (entry.j, entry.mval))).groupByKey() - val emits = rows.flatMap { - case (rowind, cols) => - cols.flatMap { - case (colind1, mval1) => - cols.map { - case (colind2, mval2) => - ((colind1, colind2), mval1 * mval2) - } - } - }.reduceByKey(_ + _) - - // Construct jblas A^T A locally - val ata = DoubleMatrix.zeros(n, n) - for (entry <- emits.collect()) { - ata.put(entry._1._1, entry._1._2, entry._2) - } - - // Since A^T A is small, we can compute its SVD directly - val svd = Singular.sparseSVD(ata) - val V = svd(0) - // This will be updated to rcond - val sigmas = MatrixFunctions.sqrt(svd(1)).toArray.filter(x => x > 1e-9) - - if (sigmas.size < k) { - throw new Exception("Not enough singular values to return k=" + k + " s=" + sigmas.size) - } - - val sigma = sigmas.take(k) - - val sc = data.sparkContext - - // prepare V for returning - val retVdata = sc.makeRDD( - Array.tabulate(V.rows, sigma.length) { - (i, j) => - MatrixEntry(i, j, V.get(i, j)) - }.flatten) - val retV = SparseMatrix(retVdata, V.rows, sigma.length) - - val retSdata = sc.makeRDD(Array.tabulate(sigma.length) { - x => MatrixEntry(x, x, sigma(x)) - }) - - val retS = SparseMatrix(retSdata, sigma.length, sigma.length) - - // Compute U as U = A V S^-1 - // turn V S^-1 into an RDD as a sparse matrix - val vsirdd = sc.makeRDD(Array.tabulate(V.rows, sigma.length) { - (i, j) => ((i, j), V.get(i, j) / sigma(j)) - }.flatten) - - if (computeU) { - // Multiply A by VS^-1 - val aCols = data.map(entry => (entry.j, (entry.i, entry.mval))) - val bRows = vsirdd.map(entry => (entry._1._1, (entry._1._2, entry._2))) - val retUdata = aCols.join(bRows).map { - case (key, ((rowInd, rowVal), (colInd, colVal))) => - ((rowInd, colInd), rowVal * colVal) - }.reduceByKey(_ + _).map { - case ((row, col), mval) => MatrixEntry(row, col, mval) - } - - val retU = SparseMatrix(retUdata, m, sigma.length) - MatrixSVD(retU, retS, retV) - } else { - MatrixSVD(null, retS, retV) - } - } -} - -/** - * Top-level methods for calling sparse Singular Value Decomposition - * NOTE: All matrices are 0-indexed - */ -object SVD { - def main(args: Array[String]) { - if (args.length < 8) { - println("Usage: SVD " + - " ") - System.exit(1) - } - - val (master, inputFile, m, n, k, output_u, output_s, output_v) = - (args(0), args(1), args(2).toInt, args(3).toInt, - args(4).toInt, args(5), args(6), args(7)) - - val sc = new SparkContext(master, "SVD") - - val rawData = sc.textFile(inputFile) - val data = rawData.map { - line => - val parts = line.split(',') - MatrixEntry(parts(0).toInt, parts(1).toInt, parts(2).toDouble) - } - - val decomposed = new SVD().setK(k).compute(SparseMatrix(data, m, n)) - val u = decomposed.U.data - val s = decomposed.S.data - val v = decomposed.V.data - - println("Computed " + s.collect().length + " singular values and vectors") - u.saveAsTextFile(output_u) - s.saveAsTextFile(output_s) - v.saveAsTextFile(output_v) - System.exit(0) - } -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala similarity index 81% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.scala rename to mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala index 2608a67bfe260..46b105457430c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala @@ -17,10 +17,5 @@ package org.apache.spark.mllib.linalg -/** - * Class that represents a row of a dense matrix - * - * @param i row index (0 indexing used) - * @param data entries of the row - */ -case class MatrixRow(val i: Int, val data: Array[Double]) +/** Represents singular value decomposition (SVD) factors. */ +case class SingularValueDecomposition[UType, VType](U: UType, s: Vector, V: VType) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala deleted file mode 100644 index b3a450e92394e..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala +++ /dev/null @@ -1,31 +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.mllib.linalg - -/** - * Class that represents the singular value decomposition of a matrix - * - * @param U such that A = USV^T is a TallSkinnyDenseMatrix - * @param S such that A = USV^T is a simple double array - * @param V such that A = USV^T, V is a 2d array matrix that holds - * singular vectors in columns. Columns are inner arrays - * i.e. V(i)(j) is standard math notation V_{ij} - */ -case class TallSkinnyMatrixSVD(val U: TallSkinnyDenseMatrix, - val S: Array[Double], - val V: Array[Array[Double]]) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala new file mode 100644 index 0000000000000..9194f657494b2 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.distributed + +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vectors + +/** + * Represents an entry in an distributed matrix. + * @param i row index + * @param j column index + * @param value value of the entry + */ +case class MatrixEntry(i: Long, j: Long, value: Double) + +/** + * Represents a matrix in coordinate format. + * + * @param entries matrix entries + * @param nRows number of rows. A non-positive value means unknown, and then the number of rows will + * be determined by the max row index plus one. + * @param nCols number of columns. A non-positive value means unknown, and then the number of + * columns will be determined by the max column index plus one. + */ +class CoordinateMatrix( + val entries: RDD[MatrixEntry], + private var nRows: Long, + private var nCols: Long) extends DistributedMatrix { + + /** Alternative constructor leaving matrix dimensions to be determined automatically. */ + def this(entries: RDD[MatrixEntry]) = this(entries, 0L, 0L) + + /** Gets or computes the number of columns. */ + override def numCols(): Long = { + if (nCols <= 0L) { + computeSize() + } + nCols + } + + /** Gets or computes the number of rows. */ + override def numRows(): Long = { + if (nRows <= 0L) { + computeSize() + } + nRows + } + + /** Converts to IndexedRowMatrix. The number of columns must be within the integer range. */ + def toIndexedRowMatrix(): IndexedRowMatrix = { + val nl = numCols() + if (nl > Int.MaxValue) { + sys.error(s"Cannot convert to a row-oriented format because the number of columns $nl is " + + "too large.") + } + val n = nl.toInt + val indexedRows = entries.map(entry => (entry.i, (entry.j.toInt, entry.value))) + .groupByKey() + .map { case (i, vectorEntries) => + IndexedRow(i, Vectors.sparse(n, vectorEntries.toSeq)) + } + new IndexedRowMatrix(indexedRows, numRows(), n) + } + + /** + * Converts to RowMatrix, dropping row indices after grouping by row index. + * The number of columns must be within the integer range. + */ + def toRowMatrix(): RowMatrix = { + toIndexedRowMatrix().toRowMatrix() + } + + /** Determines the size by computing the max row/column index. */ + private def computeSize() { + // Reduce will throw an exception if `entries` is empty. + val (m1, n1) = entries.map(entry => (entry.i, entry.j)).reduce { case ((i1, j1), (i2, j2)) => + (math.max(i1, i2), math.max(j1, j2)) + } + // There may be empty columns at the very right and empty rows at the very bottom. + nRows = math.max(nRows, m1 + 1L) + nCols = math.max(nCols, n1 + 1L) + } + + /** Collects data and assembles a local matrix. */ + private[mllib] override def toBreeze(): BDM[Double] = { + val m = numRows().toInt + val n = numCols().toInt + val mat = BDM.zeros[Double](m, n) + entries.collect().foreach { case MatrixEntry(i, j, value) => + mat(i.toInt, j.toInt) = value + } + mat + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala similarity index 60% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala rename to mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala index cbd1a2a5a4bd8..13f72a3c724ef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala @@ -15,16 +15,23 @@ * limitations under the License. */ -package org.apache.spark.mllib.linalg +package org.apache.spark.mllib.linalg.distributed -import org.apache.spark.rdd.RDD +import breeze.linalg.{DenseMatrix => BDM} +import org.apache.spark.mllib.linalg.Matrix /** - * Class that represents a sparse matrix - * - * @param data RDD of nonzero entries - * @param m number of rows - * @param n numner of columns + * Represents a distributively stored matrix backed by one or more RDDs. */ -case class SparseMatrix(val data: RDD[MatrixEntry], val m: Int, val n: Int) +trait DistributedMatrix extends Serializable { + + /** Gets or computes the number of rows. */ + def numRows(): Long + + /** Gets or computes the number of columns. */ + def numCols(): Long + + /** Collects data and assembles a local dense breeze matrix (for test only). */ + private[mllib] def toBreeze(): BDM[Double] +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala new file mode 100644 index 0000000000000..e110f070bd7c1 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.distributed + +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.linalg.SingularValueDecomposition + +/** Represents a row of [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]]. */ +case class IndexedRow(index: Long, vector: Vector) + +/** + * Represents a row-oriented [[org.apache.spark.mllib.linalg.distributed.DistributedMatrix]] with + * indexed rows. + * + * @param rows indexed rows of this matrix + * @param nRows number of rows. A non-positive value means unknown, and then the number of rows will + * be determined by the max row index plus one. + * @param nCols number of columns. A non-positive value means unknown, and then the number of + * columns will be determined by the size of the first row. + */ +class IndexedRowMatrix( + val rows: RDD[IndexedRow], + private var nRows: Long, + private var nCols: Int) extends DistributedMatrix { + + /** Alternative constructor leaving matrix dimensions to be determined automatically. */ + def this(rows: RDD[IndexedRow]) = this(rows, 0L, 0) + + override def numCols(): Long = { + if (nCols <= 0) { + // Calling `first` will throw an exception if `rows` is empty. + nCols = rows.first().vector.size + } + nCols + } + + override def numRows(): Long = { + if (nRows <= 0L) { + // Reduce will throw an exception if `rows` is empty. + nRows = rows.map(_.index).reduce(math.max) + 1L + } + nRows + } + + /** + * Drops row indices and converts this matrix to a + * [[org.apache.spark.mllib.linalg.distributed.RowMatrix]]. + */ + def toRowMatrix(): RowMatrix = { + new RowMatrix(rows.map(_.vector), 0L, nCols) + } + + /** + * Computes the singular value decomposition of this matrix. + * Denote this matrix by A (m x n), this will compute matrices U, S, V such that A = U * S * V'. + * + * There is no restriction on m, but we require `n^2` doubles to fit in memory. + * Further, n should be less than m. + + * The decomposition is computed by first computing A'A = V S^2 V', + * computing svd locally on that (since n x n is small), from which we recover S and V. + * Then we compute U via easy matrix multiplication as U = A * (V * S^-1). + * Note that this approach requires `O(n^3)` time on the master node. + * + * At most k largest non-zero singular values and associated vectors are returned. + * If there are k such values, then the dimensions of the return will be: + * + * U is an [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]] of size m x k that + * satisfies U'U = eye(k), + * s is a Vector of size k, holding the singular values in descending order, + * and V is a local Matrix of size n x k that satisfies V'V = eye(k). + * + * @param k number of singular values to keep. We might return less than k if there are + * numerically zero singular values. See rCond. + * @param computeU whether to compute U + * @param rCond the reciprocal condition number. All singular values smaller than rCond * sigma(0) + * are treated as zero, where sigma(0) is the largest singular value. + * @return SingularValueDecomposition(U, s, V) + */ + def computeSVD( + k: Int, + computeU: Boolean = false, + rCond: Double = 1e-9): SingularValueDecomposition[IndexedRowMatrix, Matrix] = { + val indices = rows.map(_.index) + val svd = toRowMatrix().computeSVD(k, computeU, rCond) + val U = if (computeU) { + val indexedRows = indices.zip(svd.U.rows).map { case (i, v) => + IndexedRow(i, v) + } + new IndexedRowMatrix(indexedRows, nRows, nCols) + } else { + null + } + SingularValueDecomposition(U, svd.s, svd.V) + } + + /** + * Multiply this matrix by a local matrix on the right. + * + * @param B a local matrix whose number of rows must match the number of columns of this matrix + * @return an IndexedRowMatrix representing the product, which preserves partitioning + */ + def multiply(B: Matrix): IndexedRowMatrix = { + val mat = toRowMatrix().multiply(B) + val indexedRows = rows.map(_.index).zip(mat.rows).map { case (i, v) => + IndexedRow(i, v) + } + new IndexedRowMatrix(indexedRows, nRows, nCols) + } + + /** + * Computes the Gramian matrix `A^T A`. + */ + def computeGramianMatrix(): Matrix = { + toRowMatrix().computeGramianMatrix() + } + + private[mllib] override def toBreeze(): BDM[Double] = { + val m = numRows().toInt + val n = numCols().toInt + val mat = BDM.zeros[Double](m, n) + rows.collect().foreach { case IndexedRow(rowIndex, vector) => + val i = rowIndex.toInt + vector.toBreeze.activeIterator.foreach { case (j, v) => + mat(i, j) = v + } + } + mat + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala new file mode 100644 index 0000000000000..f59811f18a68f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -0,0 +1,344 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.distributed + +import java.util + +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, svd => brzSvd} +import breeze.numerics.{sqrt => brzSqrt} +import com.github.fommil.netlib.BLAS.{getInstance => blas} + +import org.apache.spark.mllib.linalg._ +import org.apache.spark.rdd.RDD +import org.apache.spark.Logging + +/** + * Represents a row-oriented distributed Matrix with no meaningful row indices. + * + * @param rows rows stored as an RDD[Vector] + * @param nRows number of rows. A non-positive value means unknown, and then the number of rows will + * be determined by the number of records in the RDD `rows`. + * @param nCols number of columns. A non-positive value means unknown, and then the number of + * columns will be determined by the size of the first row. + */ +class RowMatrix( + val rows: RDD[Vector], + private var nRows: Long, + private var nCols: Int) extends DistributedMatrix with Logging { + + /** Alternative constructor leaving matrix dimensions to be determined automatically. */ + def this(rows: RDD[Vector]) = this(rows, 0L, 0) + + /** Gets or computes the number of columns. */ + override def numCols(): Long = { + if (nCols <= 0) { + // Calling `first` will throw an exception if `rows` is empty. + nCols = rows.first().size + } + nCols + } + + /** Gets or computes the number of rows. */ + override def numRows(): Long = { + if (nRows <= 0L) { + nRows = rows.count() + if (nRows == 0L) { + sys.error("Cannot determine the number of rows because it is not specified in the " + + "constructor and the rows RDD is empty.") + } + } + nRows + } + + /** + * Computes the Gramian matrix `A^T A`. + */ + def computeGramianMatrix(): Matrix = { + val n = numCols().toInt + val nt: Int = n * (n + 1) / 2 + + // Compute the upper triangular part of the gram matrix. + val GU = rows.aggregate(new BDV[Double](new Array[Double](nt)))( + seqOp = (U, v) => { + RowMatrix.dspr(1.0, v, U.data) + U + }, + combOp = (U1, U2) => U1 += U2 + ) + + RowMatrix.triuToFull(n, GU.data) + } + + /** + * Computes the singular value decomposition of this matrix. + * Denote this matrix by A (m x n), this will compute matrices U, S, V such that A = U * S * V'. + * + * There is no restriction on m, but we require `n^2` doubles to fit in memory. + * Further, n should be less than m. + + * The decomposition is computed by first computing A'A = V S^2 V', + * computing svd locally on that (since n x n is small), from which we recover S and V. + * Then we compute U via easy matrix multiplication as U = A * (V * S^-1). + * Note that this approach requires `O(n^3)` time on the master node. + * + * At most k largest non-zero singular values and associated vectors are returned. + * If there are k such values, then the dimensions of the return will be: + * + * U is a RowMatrix of size m x k that satisfies U'U = eye(k), + * s is a Vector of size k, holding the singular values in descending order, + * and V is a Matrix of size n x k that satisfies V'V = eye(k). + * + * @param k number of singular values to keep. We might return less than k if there are + * numerically zero singular values. See rCond. + * @param computeU whether to compute U + * @param rCond the reciprocal condition number. All singular values smaller than rCond * sigma(0) + * are treated as zero, where sigma(0) is the largest singular value. + * @return SingularValueDecomposition(U, s, V) + */ + def computeSVD( + k: Int, + computeU: Boolean = false, + rCond: Double = 1e-9): SingularValueDecomposition[RowMatrix, Matrix] = { + val n = numCols().toInt + require(k > 0 && k <= n, s"Request up to n singular values k=$k n=$n.") + + val G = computeGramianMatrix() + + // TODO: Use sparse SVD instead. + val (u: BDM[Double], sigmaSquares: BDV[Double], v: BDM[Double]) = + brzSvd(G.toBreeze.asInstanceOf[BDM[Double]]) + val sigmas: BDV[Double] = brzSqrt(sigmaSquares) + + // Determine effective rank. + val sigma0 = sigmas(0) + val threshold = rCond * sigma0 + var i = 0 + while (i < k && sigmas(i) >= threshold) { + i += 1 + } + val sk = i + + if (sk < k) { + logWarning(s"Requested $k singular values but only found $sk nonzeros.") + } + + val s = Vectors.dense(util.Arrays.copyOfRange(sigmas.data, 0, sk)) + val V = Matrices.dense(n, sk, util.Arrays.copyOfRange(u.data, 0, n * sk)) + + if (computeU) { + // N = Vk * Sk^{-1} + val N = new BDM[Double](n, sk, util.Arrays.copyOfRange(u.data, 0, n * sk)) + var i = 0 + var j = 0 + while (j < sk) { + i = 0 + val sigma = sigmas(j) + while (i < n) { + N(i, j) /= sigma + i += 1 + } + j += 1 + } + val U = this.multiply(Matrices.fromBreeze(N)) + SingularValueDecomposition(U, s, V) + } else { + SingularValueDecomposition(null, s, V) + } + } + + /** + * Computes the covariance matrix, treating each row as an observation. + * @return a local dense matrix of size n x n + */ + def computeCovariance(): Matrix = { + val n = numCols().toInt + + if (n > 10000) { + val mem = n * n * java.lang.Double.SIZE / java.lang.Byte.SIZE + logWarning(s"The number of columns $n is greater than 10000! " + + s"We need at least $mem bytes of memory.") + } + + val (m, mean) = rows.aggregate[(Long, BDV[Double])]((0L, BDV.zeros[Double](n)))( + seqOp = (s: (Long, BDV[Double]), v: Vector) => (s._1 + 1L, s._2 += v.toBreeze), + combOp = (s1: (Long, BDV[Double]), s2: (Long, BDV[Double])) => (s1._1 + s2._1, s1._2 += s2._2) + ) + + // Update _m if it is not set, or verify its value. + if (nRows <= 0L) { + nRows = m + } else { + require(nRows == m, + s"The number of rows $m is different from what specified or previously computed: ${nRows}.") + } + + mean :/= m.toDouble + + // We use the formula Cov(X, Y) = E[X * Y] - E[X] E[Y], which is not accurate if E[X * Y] is + // large but Cov(X, Y) is small, but it is good for sparse computation. + // TODO: find a fast and stable way for sparse data. + + val G = computeGramianMatrix().toBreeze.asInstanceOf[BDM[Double]] + + var i = 0 + var j = 0 + val m1 = m - 1.0 + var alpha = 0.0 + while (i < n) { + alpha = m / m1 * mean(i) + j = 0 + while (j < n) { + G(i, j) = G(i, j) / m1 - alpha * mean(j) + j += 1 + } + i += 1 + } + + Matrices.fromBreeze(G) + } + + /** + * Computes the top k principal components. + * Rows correspond to observations and columns correspond to variables. + * The principal components are stored a local matrix of size n-by-k. + * Each column corresponds for one principal component, + * and the columns are in descending order of component variance. + * + * @param k number of top principal components. + * @return a matrix of size n-by-k, whose columns are principal components + */ + def computePrincipalComponents(k: Int): Matrix = { + val n = numCols().toInt + require(k > 0 && k <= n, s"k = $k out of range (0, n = $n]") + + val Cov = computeCovariance().toBreeze.asInstanceOf[BDM[Double]] + + val (u: BDM[Double], _, _) = brzSvd(Cov) + + if (k == n) { + Matrices.dense(n, k, u.data) + } else { + Matrices.dense(n, k, util.Arrays.copyOfRange(u.data, 0, n * k)) + } + } + + /** + * Multiply this matrix by a local matrix on the right. + * + * @param B a local matrix whose number of rows must match the number of columns of this matrix + * @return a [[org.apache.spark.mllib.linalg.distributed.RowMatrix]] representing the product, + * which preserves partitioning + */ + def multiply(B: Matrix): RowMatrix = { + val n = numCols().toInt + require(n == B.numRows, s"Dimension mismatch: $n vs ${B.numRows}") + + require(B.isInstanceOf[DenseMatrix], + s"Only support dense matrix at this time but found ${B.getClass.getName}.") + + val Bb = rows.context.broadcast(B) + val AB = rows.mapPartitions({ iter => + val Bi = Bb.value.toBreeze.asInstanceOf[BDM[Double]] + iter.map(v => Vectors.fromBreeze(Bi.t * v.toBreeze)) + }, preservesPartitioning = true) + + new RowMatrix(AB, nRows, B.numCols) + } + + private[mllib] override def toBreeze(): BDM[Double] = { + val m = numRows().toInt + val n = numCols().toInt + val mat = BDM.zeros[Double](m, n) + var i = 0 + rows.collect().foreach { v => + v.toBreeze.activeIterator.foreach { case (j, v) => + mat(i, j) = v + } + i += 1 + } + mat + } +} + +object RowMatrix { + + /** + * Adds alpha * x * x.t to a matrix in-place. This is the same as BLAS's DSPR. + * + * @param U the upper triangular part of the matrix packed in an array (column major) + */ + private def dspr(alpha: Double, v: Vector, U: Array[Double]): Unit = { + // TODO: Find a better home (breeze?) for this method. + val n = v.size + v match { + case dv: DenseVector => + blas.dspr("U", n, 1.0, dv.values, 1, U) + case sv: SparseVector => + val indices = sv.indices + val values = sv.values + val nnz = indices.length + var colStartIdx = 0 + var prevCol = 0 + var col = 0 + var j = 0 + var i = 0 + var av = 0.0 + while (j < nnz) { + col = indices(j) + // Skip empty columns. + colStartIdx += (col - prevCol) * (col + prevCol + 1) / 2 + col = indices(j) + av = alpha * values(j) + i = 0 + while (i <= j) { + U(colStartIdx + indices(i)) += av * values(i) + i += 1 + } + j += 1 + prevCol = col + } + } + } + + /** + * Fills a full square matrix from its upper triangular part. + */ + private def triuToFull(n: Int, U: Array[Double]): Matrix = { + val G = new BDM[Double](n, n) + + var row = 0 + var col = 0 + var idx = 0 + var value = 0.0 + while (col < n) { + row = 0 + while (row < col) { + value = U(idx) + G(row, col) = value + G(col, row) = value + idx += 1 + row += 1 + } + G(col, col) = U(idx) + idx += 1 + col +=1 + } + + Matrices.dense(n, n, G.data) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala deleted file mode 100644 index 87aac347579c7..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala +++ /dev/null @@ -1,67 +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.mllib.util - -import org.apache.spark.SparkContext._ - -import org.apache.spark.mllib.linalg._ - -/** - * Helper methods for linear algebra - */ -object LAUtils { - /** - * Convert a SparseMatrix into a TallSkinnyDenseMatrix - * - * @param sp Sparse matrix to be converted - * @return dense version of the input - */ - def sparseToTallSkinnyDense(sp: SparseMatrix): TallSkinnyDenseMatrix = { - val m = sp.m - val n = sp.n - val rows = sp.data.map(x => (x.i, (x.j, x.mval))).groupByKey().map { - case (i, cols) => - val rowArray = Array.ofDim[Double](n) - var j = 0 - val colsItr = cols.iterator - while (colsItr.hasNext) { - val element = colsItr.next - rowArray(element._1) = element._2 - j += 1 - } - MatrixRow(i, rowArray) - } - TallSkinnyDenseMatrix(rows, m, n) - } - - /** - * Convert a TallSkinnyDenseMatrix to a SparseMatrix - * - * @param a matrix to be converted - * @return sparse version of the input - */ - def denseToSparse(a: TallSkinnyDenseMatrix): SparseMatrix = { - val m = a.m - val n = a.n - val data = a.rows.flatMap { - mrow => Array.tabulate(n)(j => MatrixEntry(mrow.i, j, mrow.data(j))) - .filter(x => x.mval != 0) - } - SparseMatrix(data, m, n) - } -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala similarity index 51% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala rename to mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala index 416996fcbe760..82d49c76ed02b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala @@ -17,11 +17,24 @@ package org.apache.spark.mllib.linalg -/** - * Class that represents an entry in a sparse matrix of doubles. - * - * @param i row index (0 indexing used) - * @param j column index (0 indexing used) - * @param mval value of entry in matrix - */ -case class MatrixEntry(val i: Int, val j: Int, val mval: Double) +import org.scalatest.FunSuite + +import breeze.linalg.{DenseMatrix => BDM} + +class BreezeMatrixConversionSuite extends FunSuite { + test("dense matrix to breeze") { + val mat = Matrices.dense(3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val breeze = mat.toBreeze.asInstanceOf[BDM[Double]] + assert(breeze.rows === mat.numRows) + assert(breeze.cols === mat.numCols) + assert(breeze.data.eq(mat.asInstanceOf[DenseMatrix].values), "should not copy data") + } + + test("dense breeze matrix to matrix") { + val breeze = new BDM[Double](3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val mat = Matrices.fromBreeze(breeze).asInstanceOf[DenseMatrix] + assert(mat.numRows === breeze.rows) + assert(mat.numCols === breeze.cols) + assert(mat.values.eq(breeze.data), "should not copy data") + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala similarity index 58% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala rename to mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index e4ef3c58e8680..9c66b4db9f16b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -17,14 +17,23 @@ package org.apache.spark.mllib.linalg -import org.apache.spark.rdd.RDD +import org.scalatest.FunSuite +class MatricesSuite extends FunSuite { + test("dense matrix construction") { + val m = 3 + val n = 2 + val values = Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0) + val mat = Matrices.dense(m, n, values).asInstanceOf[DenseMatrix] + assert(mat.numRows === m) + assert(mat.numCols === n) + assert(mat.values.eq(values), "should not copy data") + assert(mat.toArray.eq(values), "toArray should not copy data") + } -/** - * Class that represents a dense matrix - * - * @param rows RDD of rows - * @param m number of rows - * @param n number of columns - */ -case class TallSkinnyDenseMatrix(val rows: RDD[MatrixRow], val m: Int, val n: Int) + test("dense matrix construction with wrong dimension") { + intercept[RuntimeException] { + Matrices.dense(3, 2, Array(0.0, 1.0, 2.0)) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala deleted file mode 100644 index 5e5086b1bf73e..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala +++ /dev/null @@ -1,124 +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.mllib.linalg - -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - -import org.apache.spark.mllib.util._ - -import org.jblas._ - -class PCASuite extends FunSuite with BeforeAndAfterAll { - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - val EPSILON = 1e-3 - - // Return jblas matrix from sparse matrix RDD - def getDenseMatrix(matrix: SparseMatrix) : DoubleMatrix = { - val data = matrix.data - val ret = DoubleMatrix.zeros(matrix.m, matrix.n) - matrix.data.collect().map(x => ret.put(x.i, x.j, x.mval)) - ret - } - - def assertMatrixApproximatelyEquals(a: DoubleMatrix, b: DoubleMatrix) { - assert(a.rows == b.rows && a.columns == b.columns, - "dimension mismatch: $a.rows vs $b.rows and $a.columns vs $b.columns") - for (i <- 0 until a.columns) { - val aCol = a.getColumn(i) - val bCol = b.getColumn(i) - val diff = Math.min(aCol.sub(bCol).norm1, aCol.add(bCol).norm1) - assert(diff < EPSILON, "matrix mismatch: " + diff) - } - } - - test("full rank matrix pca") { - val m = 5 - val n = 3 - val dataArr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, Math.sin(a + b + a * b)) }.flatten - val data = sc.makeRDD(dataArr, 3) - val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) - - val realPCAArray = Array((0,0,-0.2579), (0,1,-0.6602), (0,2,0.7054), - (1,0,-0.1448), (1,1,0.7483), (1,2,0.6474), - (2,0,0.9553), (2,1,-0.0649), (2,2,0.2886)) - val realPCA = sc.makeRDD(realPCAArray.map(x => MatrixEntry(x._1, x._2, x._3)), 3) - - val coeffs = new DoubleMatrix(new PCA().setK(n).compute(a)) - - assertMatrixApproximatelyEquals(getDenseMatrix(SparseMatrix(realPCA,n,n)), coeffs) - } - - test("sparse matrix full rank matrix pca") { - val m = 5 - val n = 3 - // the entry that gets dropped is zero to test sparse support - val dataArr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, Math.sin(a + b + a * b)) }.flatten.drop(1) - val data = sc.makeRDD(dataArr, 3) - val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) - - val realPCAArray = Array((0,0,-0.2579), (0,1,-0.6602), (0,2,0.7054), - (1,0,-0.1448), (1,1,0.7483), (1,2,0.6474), - (2,0,0.9553), (2,1,-0.0649), (2,2,0.2886)) - val realPCA = sc.makeRDD(realPCAArray.map(x => MatrixEntry(x._1, x._2, x._3))) - - val coeffs = new DoubleMatrix(new PCA().setK(n).compute(a)) - - assertMatrixApproximatelyEquals(getDenseMatrix(SparseMatrix(realPCA,n,n)), coeffs) - } - - test("truncated matrix pca") { - val m = 5 - val n = 3 - val dataArr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, Math.sin(a + b + a * b)) }.flatten - - val data = sc.makeRDD(dataArr, 3) - val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) - - val realPCAArray = Array((0,0,-0.2579), (0,1,-0.6602), - (1,0,-0.1448), (1,1,0.7483), - (2,0,0.9553), (2,1,-0.0649)) - val realPCA = sc.makeRDD(realPCAArray.map(x => MatrixEntry(x._1, x._2, x._3))) - - val k = 2 - val coeffs = new DoubleMatrix(new PCA().setK(k).compute(a)) - - assertMatrixApproximatelyEquals(getDenseMatrix(SparseMatrix(realPCA,n,k)), coeffs) - } -} - - 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 deleted file mode 100644 index 20e2b0f84be06..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala +++ /dev/null @@ -1,194 +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.mllib.linalg - -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import org.jblas.{DoubleMatrix, Singular, MatrixFunctions} - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - -import org.apache.spark.mllib.util._ - -import org.jblas._ - -class SVDSuite extends FunSuite with BeforeAndAfterAll { - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - val EPSILON = 1e-4 - - // Return jblas matrix from sparse matrix RDD - def getDenseMatrix(matrix: SparseMatrix) : DoubleMatrix = { - val data = matrix.data - val m = matrix.m - val n = matrix.n - val ret = DoubleMatrix.zeros(m, n) - matrix.data.collect().map(x => ret.put(x.i, x.j, x.mval)) - ret - } - - def assertMatrixApproximatelyEquals(a: DoubleMatrix, b: DoubleMatrix) { - assert(a.rows == b.rows && a.columns == b.columns, - "dimension mismatch: $a.rows vs $b.rows and $a.columns vs $b.columns") - for (i <- 0 until a.columns) { - val aCol = a.getColumn(i) - val bCol = b.getColumn(i) - val diff = Math.min(aCol.sub(bCol).norm1, aCol.add(bCol).norm1) - assert(diff < EPSILON, "matrix mismatch: " + diff) - } - } - - test("full rank matrix svd") { - val m = 10 - val n = 3 - val datarr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, (a + 2).toDouble * (b + 1) / (1 + a + b)) }.flatten - val data = sc.makeRDD(datarr, 3) - - val a = SparseMatrix(data, m, n) - - val decomposed = new SVD().setK(n).compute(a) - val u = decomposed.U - val v = decomposed.V - val s = decomposed.S - - val denseA = getDenseMatrix(a) - val svd = Singular.sparseSVD(denseA) - - val retu = getDenseMatrix(u) - val rets = getDenseMatrix(s) - val retv = getDenseMatrix(v) - - - // check individual decomposition - assertMatrixApproximatelyEquals(retu, svd(0)) - assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1))) - assertMatrixApproximatelyEquals(retv, svd(2)) - - // check multiplication guarantee - assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) - } - - test("dense full rank matrix svd") { - val m = 10 - val n = 3 - val datarr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, (a + 2).toDouble * (b + 1) / (1 + a + b)) }.flatten - val data = sc.makeRDD(datarr, 3) - - val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) - - val decomposed = new SVD().setK(n).setComputeU(true).compute(a) - val u = LAUtils.denseToSparse(decomposed.U) - val v = decomposed.V - val s = decomposed.S - - val denseA = getDenseMatrix(LAUtils.denseToSparse(a)) - val svd = Singular.sparseSVD(denseA) - - val retu = getDenseMatrix(u) - val rets = DoubleMatrix.diag(new DoubleMatrix(s)) - val retv = new DoubleMatrix(v) - - - // check individual decomposition - assertMatrixApproximatelyEquals(retu, svd(0)) - assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1))) - assertMatrixApproximatelyEquals(retv, svd(2)) - - // check multiplication guarantee - assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) - } - - test("rank one matrix svd") { - val m = 10 - val n = 3 - val data = sc.makeRDD(Array.tabulate(m, n){ (a,b) => - MatrixEntry(a, b, 1.0) }.flatten ) - val k = 1 - - val a = SparseMatrix(data, m, n) - - val decomposed = new SVD().setK(k).compute(a) - val u = decomposed.U - val s = decomposed.S - val v = decomposed.V - val retrank = s.data.collect().length - - assert(retrank == 1, "rank returned not one") - - val denseA = getDenseMatrix(a) - val svd = Singular.sparseSVD(denseA) - - val retu = getDenseMatrix(u) - val rets = getDenseMatrix(s) - val retv = getDenseMatrix(v) - - // check individual decomposition - assertMatrixApproximatelyEquals(retu, svd(0).getColumn(0)) - assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1).getRow(0))) - assertMatrixApproximatelyEquals(retv, svd(2).getColumn(0)) - - // check multiplication guarantee - assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) - } - - test("truncated with k") { - val m = 10 - val n = 3 - val data = sc.makeRDD(Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, (a + 2).toDouble * (b + 1)/(1 + a + b)) }.flatten ) - val a = SparseMatrix(data, m, n) - - val k = 1 // only one svalue above this - - val decomposed = new SVD().setK(k).compute(a) - val u = decomposed.U - val s = decomposed.S - val v = decomposed.V - val retrank = s.data.collect().length - - val denseA = getDenseMatrix(a) - val svd = Singular.sparseSVD(denseA) - - val retu = getDenseMatrix(u) - val rets = getDenseMatrix(s) - val retv = getDenseMatrix(v) - - assert(retrank == 1, "rank returned not one") - - // check individual decomposition - assertMatrixApproximatelyEquals(retu, svd(0).getColumn(0)) - assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1).getRow(0))) - assertMatrixApproximatelyEquals(retv, svd(2).getColumn(0)) - } -} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala new file mode 100644 index 0000000000000..cd45438fb628f --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.distributed + +import org.scalatest.FunSuite + +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.linalg.Vectors + +class CoordinateMatrixSuite extends FunSuite with LocalSparkContext { + + val m = 5 + val n = 4 + var mat: CoordinateMatrix = _ + + override def beforeAll() { + super.beforeAll() + val entries = sc.parallelize(Seq( + (0, 0, 1.0), + (0, 1, 2.0), + (1, 1, 3.0), + (1, 2, 4.0), + (2, 2, 5.0), + (2, 3, 6.0), + (3, 0, 7.0), + (3, 3, 8.0), + (4, 1, 9.0)), 3).map { case (i, j, value) => + MatrixEntry(i, j, value) + } + mat = new CoordinateMatrix(entries) + } + + test("size") { + assert(mat.numRows() === m) + assert(mat.numCols() === n) + } + + test("empty entries") { + val entries = sc.parallelize(Seq[MatrixEntry](), 1) + val emptyMat = new CoordinateMatrix(entries) + intercept[RuntimeException] { + emptyMat.numCols() + } + intercept[RuntimeException] { + emptyMat.numRows() + } + } + + test("toBreeze") { + val expected = BDM( + (1.0, 2.0, 0.0, 0.0), + (0.0, 3.0, 4.0, 0.0), + (0.0, 0.0, 5.0, 6.0), + (7.0, 0.0, 0.0, 8.0), + (0.0, 9.0, 0.0, 0.0)) + assert(mat.toBreeze() === expected) + } + + test("toIndexedRowMatrix") { + val indexedRowMatrix = mat.toIndexedRowMatrix() + val expected = BDM( + (1.0, 2.0, 0.0, 0.0), + (0.0, 3.0, 4.0, 0.0), + (0.0, 0.0, 5.0, 6.0), + (7.0, 0.0, 0.0, 8.0), + (0.0, 9.0, 0.0, 0.0)) + assert(indexedRowMatrix.toBreeze() === expected) + } + + test("toRowMatrix") { + val rowMatrix = mat.toRowMatrix() + val rows = rowMatrix.rows.collect().toSet + val expected = Set( + Vectors.dense(1.0, 2.0, 0.0, 0.0), + Vectors.dense(0.0, 3.0, 4.0, 0.0), + Vectors.dense(0.0, 0.0, 5.0, 6.0), + Vectors.dense(7.0, 0.0, 0.0, 8.0), + Vectors.dense(0.0, 9.0, 0.0, 0.0)) + assert(rows === expected) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala new file mode 100644 index 0000000000000..f7c46f23b746d --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.distributed + +import org.scalatest.FunSuite + +import breeze.linalg.{diag => brzDiag, DenseMatrix => BDM, DenseVector => BDV} + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.{Matrices, Vectors} + +class IndexedRowMatrixSuite extends FunSuite with LocalSparkContext { + + val m = 4 + val n = 3 + val data = Seq( + (0L, Vectors.dense(0.0, 1.0, 2.0)), + (1L, Vectors.dense(3.0, 4.0, 5.0)), + (3L, Vectors.dense(9.0, 0.0, 1.0)) + ).map(x => IndexedRow(x._1, x._2)) + var indexedRows: RDD[IndexedRow] = _ + + override def beforeAll() { + super.beforeAll() + indexedRows = sc.parallelize(data, 2) + } + + test("size") { + val mat1 = new IndexedRowMatrix(indexedRows) + assert(mat1.numRows() === m) + assert(mat1.numCols() === n) + + val mat2 = new IndexedRowMatrix(indexedRows, 5, 0) + assert(mat2.numRows() === 5) + assert(mat2.numCols() === n) + } + + test("empty rows") { + val rows = sc.parallelize(Seq[IndexedRow](), 1) + val mat = new IndexedRowMatrix(rows) + intercept[RuntimeException] { + mat.numRows() + } + intercept[RuntimeException] { + mat.numCols() + } + } + + test("toBreeze") { + val mat = new IndexedRowMatrix(indexedRows) + val expected = BDM( + (0.0, 1.0, 2.0), + (3.0, 4.0, 5.0), + (0.0, 0.0, 0.0), + (9.0, 0.0, 1.0)) + assert(mat.toBreeze() === expected) + } + + test("toRowMatrix") { + val idxRowMat = new IndexedRowMatrix(indexedRows) + val rowMat = idxRowMat.toRowMatrix() + assert(rowMat.numCols() === n) + assert(rowMat.numRows() === 3, "should drop empty rows") + assert(rowMat.rows.collect().toSeq === data.map(_.vector).toSeq) + } + + test("multiply a local matrix") { + val A = new IndexedRowMatrix(indexedRows) + val B = Matrices.dense(3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val C = A.multiply(B) + val localA = A.toBreeze() + val localC = C.toBreeze() + val expected = localA * B.toBreeze.asInstanceOf[BDM[Double]] + assert(localC === expected) + } + + test("gram") { + val A = new IndexedRowMatrix(indexedRows) + val G = A.computeGramianMatrix() + val expected = BDM( + (90.0, 12.0, 24.0), + (12.0, 17.0, 22.0), + (24.0, 22.0, 30.0)) + assert(G.toBreeze === expected) + } + + test("svd") { + val A = new IndexedRowMatrix(indexedRows) + val svd = A.computeSVD(n, computeU = true) + assert(svd.U.isInstanceOf[IndexedRowMatrix]) + val localA = A.toBreeze() + val U = svd.U.toBreeze() + val s = svd.s.toBreeze.asInstanceOf[BDV[Double]] + val V = svd.V.toBreeze.asInstanceOf[BDM[Double]] + assert(closeToZero(U.t * U - BDM.eye[Double](n))) + assert(closeToZero(V.t * V - BDM.eye[Double](n))) + assert(closeToZero(U * brzDiag(s) * V.t - localA)) + } + + def closeToZero(G: BDM[Double]): Boolean = { + G.valuesIterator.map(math.abs).sum < 1e-6 + } +} + diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala new file mode 100644 index 0000000000000..71ee8e8a4f6fd --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.distributed + +import org.scalatest.FunSuite + +import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, norm => brzNorm, svd => brzSvd} + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.linalg.{Matrices, Vectors, Vector} + +class RowMatrixSuite extends FunSuite with LocalSparkContext { + + val m = 4 + val n = 3 + val arr = Array(0.0, 3.0, 6.0, 9.0, 1.0, 4.0, 7.0, 0.0, 2.0, 5.0, 8.0, 1.0) + val denseData = Seq( + Vectors.dense(0.0, 1.0, 2.0), + Vectors.dense(3.0, 4.0, 5.0), + Vectors.dense(6.0, 7.0, 8.0), + Vectors.dense(9.0, 0.0, 1.0) + ) + val sparseData = Seq( + Vectors.sparse(3, Seq((1, 1.0), (2, 2.0))), + Vectors.sparse(3, Seq((0, 3.0), (1, 4.0), (2, 5.0))), + Vectors.sparse(3, Seq((0, 6.0), (1, 7.0), (2, 8.0))), + Vectors.sparse(3, Seq((0, 9.0), (2, 1.0))) + ) + + val principalComponents = BDM( + (0.0, 1.0, 0.0), + (math.sqrt(2.0) / 2.0, 0.0, math.sqrt(2.0) / 2.0), + (math.sqrt(2.0) / 2.0, 0.0, - math.sqrt(2.0) / 2.0)) + + var denseMat: RowMatrix = _ + var sparseMat: RowMatrix = _ + + override def beforeAll() { + super.beforeAll() + denseMat = new RowMatrix(sc.parallelize(denseData, 2)) + sparseMat = new RowMatrix(sc.parallelize(sparseData, 2)) + } + + test("size") { + assert(denseMat.numRows() === m) + assert(denseMat.numCols() === n) + assert(sparseMat.numRows() === m) + assert(sparseMat.numCols() === n) + } + + test("empty rows") { + val rows = sc.parallelize(Seq[Vector](), 1) + val emptyMat = new RowMatrix(rows) + intercept[RuntimeException] { + emptyMat.numCols() + } + intercept[RuntimeException] { + emptyMat.numRows() + } + } + + test("toBreeze") { + val expected = BDM( + (0.0, 1.0, 2.0), + (3.0, 4.0, 5.0), + (6.0, 7.0, 8.0), + (9.0, 0.0, 1.0)) + for (mat <- Seq(denseMat, sparseMat)) { + assert(mat.toBreeze() === expected) + } + } + + test("gram") { + val expected = + Matrices.dense(n, n, Array(126.0, 54.0, 72.0, 54.0, 66.0, 78.0, 72.0, 78.0, 94.0)) + for (mat <- Seq(denseMat, sparseMat)) { + val G = mat.computeGramianMatrix() + assert(G.toBreeze === expected.toBreeze) + } + } + + test("svd of a full-rank matrix") { + for (mat <- Seq(denseMat, sparseMat)) { + val localMat = mat.toBreeze() + val (localU, localSigma, localVt) = brzSvd(localMat) + val localV: BDM[Double] = localVt.t.toDenseMatrix + for (k <- 1 to n) { + val svd = mat.computeSVD(k, computeU = true) + val U = svd.U + val s = svd.s + val V = svd.V + assert(U.numRows() === m) + assert(U.numCols() === k) + assert(s.size === k) + assert(V.numRows === n) + assert(V.numCols === k) + assertColumnEqualUpToSign(U.toBreeze(), localU, k) + assertColumnEqualUpToSign(V.toBreeze.asInstanceOf[BDM[Double]], localV, k) + assert(closeToZero(s.toBreeze.asInstanceOf[BDV[Double]] - localSigma(0 until k))) + } + val svdWithoutU = mat.computeSVD(n) + assert(svdWithoutU.U === null) + } + } + + test("svd of a low-rank matrix") { + val rows = sc.parallelize(Array.fill(4)(Vectors.dense(1.0, 1.0)), 2) + val mat = new RowMatrix(rows, 4, 2) + val svd = mat.computeSVD(2, computeU = true) + assert(svd.s.size === 1, "should not return zero singular values") + assert(svd.U.numRows() === 4) + assert(svd.U.numCols() === 1) + assert(svd.V.numRows === 2) + assert(svd.V.numCols === 1) + } + + def closeToZero(G: BDM[Double]): Boolean = { + G.valuesIterator.map(math.abs).sum < 1e-6 + } + + def closeToZero(v: BDV[Double]): Boolean = { + brzNorm(v, 1.0) < 1e-6 + } + + def assertColumnEqualUpToSign(A: BDM[Double], B: BDM[Double], k: Int) { + assert(A.rows === B.rows) + for (j <- 0 until k) { + val aj = A(::, j) + val bj = B(::, j) + assert(closeToZero(aj - bj) || closeToZero(aj + bj), + s"The $j-th columns mismatch: $aj and $bj") + } + } + + test("pca") { + for (mat <- Seq(denseMat, sparseMat); k <- 1 to n) { + val pc = denseMat.computePrincipalComponents(k) + assert(pc.numRows === n) + assert(pc.numCols === k) + assertColumnEqualUpToSign(pc.toBreeze.asInstanceOf[BDM[Double]], principalComponents, k) + } + } + + test("multiply a local matrix") { + val B = Matrices.dense(n, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + for (mat <- Seq(denseMat, sparseMat)) { + val AB = mat.multiply(B) + assert(AB.numRows() === m) + assert(AB.numCols() === 2) + assert(AB.rows.collect().toSeq === Seq( + Vectors.dense(5.0, 14.0), + Vectors.dense(14.0, 50.0), + Vectors.dense(23.0, 86.0), + Vectors.dense(2.0, 32.0) + )) + } + } +} From 87bd1f9ef7d547ee54a8a83214b45462e0751efb Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 9 Apr 2014 01:14:46 -0700 Subject: [PATCH 036/641] SPARK-1093: Annotate developer and experimental API's This patch marks some existing classes as private[spark] and adds two types of API annotations: - `EXPERIMENTAL API` = experimental user-facing module - `DEVELOPER API - UNSTABLE` = developer-facing API that might change There is some discussion of the different mechanisms for doing this here: https://issues.apache.org/jira/browse/SPARK-1081 I was pretty aggressive with marking things private. Keep in mind that if we want to open something up in the future we can, but we can never reduce visibility. A few notes here: - In the past we've been inconsistent with the visiblity of the X-RDD classes. This patch marks them private whenever there is an existing function in RDD that can directly creat them (e.g. CoalescedRDD and rdd.coalesce()). One trade-off here is users can't subclass them. - Noted that compression and serialization formats don't have to be wire compatible across versions. - Compression codecs and serialization formats are semi-private as users typically don't instantiate them directly. - Metrics sources are made private - user only interacts with them through Spark's reflection Author: Patrick Wendell Author: Andrew Or Closes #274 from pwendell/private-apis and squashes the following commits: 44179e4 [Patrick Wendell] Merge remote-tracking branch 'apache-github/master' into private-apis 042c803 [Patrick Wendell] spark.annotations -> spark.annotation bfe7b52 [Patrick Wendell] Adding experimental for approximate counts 8d0c873 [Patrick Wendell] Warning in SparkEnv 99b223a [Patrick Wendell] Cleaning up annotations e849f64 [Patrick Wendell] Merge pull request #2 from andrewor14/annotations 982a473 [Andrew Or] Generalize jQuery matching for non Spark-core API docs a01c076 [Patrick Wendell] Merge pull request #1 from andrewor14/annotations c1bcb41 [Andrew Or] DeveloperAPI -> DeveloperApi 0d48908 [Andrew Or] Comments and new lines (minor) f3954e0 [Andrew Or] Add identifier tags in comments to work around scaladocs bug 99192ef [Andrew Or] Dynamically add badges based on annotations 824011b [Andrew Or] Add support for injecting arbitrary JavaScript to API docs 037755c [Patrick Wendell] Some changes after working with andrew or f7d124f [Patrick Wendell] Small fixes c318b24 [Patrick Wendell] Use CSS styles e4c76b9 [Patrick Wendell] Logging f390b13 [Patrick Wendell] Better visibility for workaround constructors d6b0afd [Patrick Wendell] Small chang to existing constructor 403ba52 [Patrick Wendell] Style fix 870a7ba [Patrick Wendell] Work around for SI-8479 7fb13b2 [Patrick Wendell] Changes to UnionRDD and EmptyRDD 4a9e90c [Patrick Wendell] EXPERIMENTAL API --> EXPERIMENTAL c581dce [Patrick Wendell] Changes after building against Shark. 8452309 [Patrick Wendell] Style fixes 1ed27d2 [Patrick Wendell] Formatting and coloring of badges cd7a465 [Patrick Wendell] Code review feedback 2f706f1 [Patrick Wendell] Don't use floats 542a736 [Patrick Wendell] Small fixes cf23ec6 [Patrick Wendell] Marking GraphX as alpha d86818e [Patrick Wendell] Another naming change 5a76ed6 [Patrick Wendell] More visiblity clean-up 42c1f09 [Patrick Wendell] Using better labels 9d48cbf [Patrick Wendell] Initial pass --- .../scala/org/apache/spark/Aggregator.scala | 3 + .../scala/org/apache/spark/Dependency.scala | 11 +++ .../scala/org/apache/spark/FutureAction.scala | 7 ++ .../apache/spark/InterruptibleIterator.scala | 2 +- .../main/scala/org/apache/spark/Logging.scala | 7 ++ .../apache/spark/SerializableWritable.scala | 3 + .../scala/org/apache/spark/SparkContext.scala | 86 ++++++++++++++++--- .../scala/org/apache/spark/SparkEnv.scala | 8 +- .../scala/org/apache/spark/TaskContext.scala | 6 ++ .../org/apache/spark/TaskEndReason.scala | 31 +++++-- .../spark/annotation/AlphaComponent.java | 26 ++++++ .../apache/spark/annotation/DeveloperApi.java | 30 +++++++ .../apache/spark/annotation/Experimental.java | 31 +++++++ .../apache/spark/api/java/JavaDoubleRDD.scala | 19 +++- .../apache/spark/api/java/JavaPairRDD.scala | 11 ++- .../apache/spark/api/java/JavaRDDLike.scala | 9 +- .../spark/api/java/JavaSparkContext.scala | 2 +- .../org/apache/spark/api/java/JavaUtils.scala | 2 +- .../spark/broadcast/BroadcastFactory.scala | 7 +- .../spark/executor/ExecutorSource.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 20 ++++- .../apache/spark/io/CompressionCodec.scala | 19 ++++ .../spark/metrics/sink/ConsoleSink.scala | 2 +- .../apache/spark/metrics/sink/CsvSink.scala | 2 +- .../spark/metrics/sink/GraphiteSink.scala | 2 +- .../apache/spark/metrics/sink/JmxSink.scala | 2 +- .../spark/metrics/sink/MetricsServlet.scala | 2 +- .../org/apache/spark/metrics/sink/Sink.scala | 2 +- .../spark/metrics/source/JvmSource.scala | 2 +- .../apache/spark/metrics/source/Source.scala | 2 +- .../main/scala/org/apache/spark/package.scala | 9 ++ .../apache/spark/partial/BoundedDouble.scala | 6 +- .../apache/spark/partial/PartialResult.scala | 3 + .../apache/spark/rdd/AsyncRDDActions.scala | 3 + .../org/apache/spark/rdd/CoGroupedRDD.scala | 8 +- .../org/apache/spark/rdd/CoalescedRDD.scala | 4 +- .../apache/spark/rdd/DoubleRDDFunctions.scala | 13 ++- .../scala/org/apache/spark/rdd/EmptyRDD.scala | 4 +- .../org/apache/spark/rdd/HadoopRDD.scala | 6 ++ .../scala/org/apache/spark/rdd/JdbcRDD.scala | 2 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 6 ++ .../apache/spark/rdd/PairRDDFunctions.scala | 5 +- .../spark/rdd/PartitionPruningRDD.scala | 9 +- .../spark/rdd/PartitionwiseSampledRDD.scala | 2 +- .../scala/org/apache/spark/rdd/PipedRDD.scala | 2 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 29 ++++++- .../org/apache/spark/rdd/SampledRDD.scala | 2 +- .../org/apache/spark/rdd/ShuffledRDD.scala | 3 + .../scala/org/apache/spark/rdd/UnionRDD.scala | 2 + .../spark/rdd/ZippedPartitionsRDD.scala | 8 +- .../org/apache/spark/rdd/ZippedRDD.scala | 2 +- .../spark/scheduler/InputFormatInfo.scala | 3 + .../apache/spark/scheduler/JobLogger.scala | 4 +- .../apache/spark/scheduler/JobResult.scala | 10 ++- .../spark/scheduler/SparkListener.scala | 20 ++++- .../apache/spark/scheduler/SplitInfo.scala | 3 + .../apache/spark/scheduler/StageInfo.scala | 4 +- .../org/apache/spark/scheduler/TaskInfo.scala | 15 ++-- .../apache/spark/scheduler/TaskLocality.scala | 5 +- .../spark/serializer/JavaSerializer.scala | 7 ++ .../spark/serializer/KryoSerializer.scala | 4 + .../apache/spark/serializer/Serializer.scala | 15 +++- .../apache/spark/storage/BlockManager.scala | 8 +- .../apache/spark/storage/StorageUtils.scala | 2 + .../spark/util/BoundedPriorityQueue.scala | 2 +- .../apache/spark/util/CollectionsUtil.scala | 2 +- .../org/apache/spark/util/Distribution.scala | 4 +- .../org/apache/spark/util/FileLogger.scala | 2 +- .../org/apache/spark/util/MutablePair.scala | 4 + .../spark/util/TimeStampedHashSet.scala | 2 +- .../scala/org/apache/spark/util/Vector.scala | 1 + .../spark/util/collection/AppendOnlyMap.scala | 9 +- .../collection/ExternalAppendOnlyMap.scala | 6 +- .../spark/util/collection/OpenHashMap.scala | 5 +- .../spark/util/random/Pseudorandom.scala | 4 + .../spark/util/random/RandomSampler.scala | 8 ++ docs/_plugins/copy_api_dirs.rb | 24 +++--- docs/css/api-docs.css | 18 ++++ docs/js/api-docs.js | 26 ++++++ .../org/apache/spark/graphx/package.scala | 5 +- .../spark/sql/catalyst/expressions/Row.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 9 +- .../org/apache/spark/sql/SchemaRDD.scala | 21 ++--- .../spark/tools}/StoragePerfTester.scala | 4 +- 84 files changed, 614 insertions(+), 130 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java create mode 100644 core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java create mode 100644 core/src/main/scala/org/apache/spark/annotation/Experimental.java create mode 100644 docs/css/api-docs.css create mode 100644 docs/js/api-docs.js rename {core/src/main/scala/org/apache/spark/storage => tools/src/main/scala/org/apache/spark/tools}/StoragePerfTester.scala (97%) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index ceead59b79ed6..59fdf659c9e11 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,15 +17,18 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** + * :: DeveloperApi :: * A set of functions used to aggregate data. * * @param createCombiner function to create the initial value of the aggregation. * @param mergeValue function to merge a new value into the aggregation result. * @param mergeCombiners function to merge outputs from multiple mergeValue function. */ +@DeveloperApi case class Aggregator[K, V, C] ( createCombiner: V => C, mergeValue: (C, V) => C, diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 1cd629c15bd46..2c31cc20211ff 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -17,19 +17,24 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer /** + * :: DeveloperApi :: * Base class for dependencies. */ +@DeveloperApi abstract class Dependency[T](val rdd: RDD[T]) extends Serializable /** + * :: DeveloperApi :: * Base class for dependencies where each partition of the parent RDD is used by at most one * partition of the child RDD. Narrow dependencies allow for pipelined execution. */ +@DeveloperApi abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** * Get the parent partitions for a child partition. @@ -41,6 +46,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** + * :: DeveloperApi :: * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output @@ -48,6 +54,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { * the default serializer, as specified by `spark.serializer` config option, will * be used. */ +@DeveloperApi class ShuffleDependency[K, V]( @transient rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, @@ -61,20 +68,24 @@ class ShuffleDependency[K, V]( /** + * :: DeveloperApi :: * Represents a one-to-one dependency between partitions of the parent and child RDDs. */ +@DeveloperApi class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { override def getParents(partitionId: Int) = List(partitionId) } /** + * :: DeveloperApi :: * Represents a one-to-one dependency between ranges of partitions in the parent and child RDDs. * @param rdd the parent RDD * @param inStart the start of the range in the parent RDD * @param outStart the start of the range in the child RDD * @param length the length of the range */ +@DeveloperApi class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int) extends NarrowDependency[T](rdd) { diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 2eec09cd1c795..1e4dec86a0530 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -21,13 +21,16 @@ import scala.concurrent._ import scala.concurrent.duration.Duration import scala.util.Try +import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter} /** + * :: Experimental :: * A future for the result of an action to support cancellation. This is an extension of the * Scala Future interface to support cancellation. */ +@Experimental trait FutureAction[T] extends Future[T] { // Note that we redefine methods of the Future trait here explicitly so we can specify a different // documentation (with reference to the word "action"). @@ -84,9 +87,11 @@ trait FutureAction[T] extends Future[T] { /** + * :: Experimental :: * A [[FutureAction]] holding the result of an action that triggers a single job. Examples include * count, collect, reduce. */ +@Experimental class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: => T) extends FutureAction[T] { @@ -148,10 +153,12 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: /** + * :: Experimental :: * A [[FutureAction]] for actions that could trigger multiple Spark jobs. Examples include take, * takeSample. Cancellation works by setting the cancelled flag to true and interrupting the * action thread if it is being blocked by a job. */ +@Experimental class ComplexFutureAction[T] extends FutureAction[T] { // Pointer to the thread that is executing the action. It is set when the action is run. diff --git a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala index 9b1601d5b95fa..fd1802ba2f984 100644 --- a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala +++ b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala @@ -21,7 +21,7 @@ package org.apache.spark * An iterator that wraps around an existing iterator to provide task killing functionality. * It works by checking the interrupted flag in [[TaskContext]]. */ -class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T]) +private[spark] class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T]) extends Iterator[T] { def hasNext: Boolean = !context.interrupted && delegate.hasNext diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 7423082e34f47..e5e15617acb10 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -21,11 +21,18 @@ import org.apache.log4j.{LogManager, PropertyConfigurator} import org.slf4j.{Logger, LoggerFactory} import org.slf4j.impl.StaticLoggerBinder +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows * logging messages at different levels using methods that only evaluate parameters lazily if the * log level is enabled. + * + * NOTE: DO NOT USE this class outside of Spark. It is intended as an internal utility. + * This will likely be changed or removed in future releases. */ +@DeveloperApi trait Logging { // Make the log field transient so that objects with Logging can // be serialized and used on another machine diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index dff665cae6cb6..e50b9ac2291f9 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -23,6 +23,9 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.ObjectWritable import org.apache.hadoop.io.Writable +import org.apache.spark.annotation.DeveloperApi + +@DeveloperApi class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable { def value = t override def toString = t.toString diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d7124616d3bfb..f7750514ae13d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -34,6 +34,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHad import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.mesos.MesosNativeLibrary +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.input.WholeTextFileInputFormat @@ -48,22 +49,35 @@ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} /** + * :: DeveloperApi :: * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. * * @param config a Spark Config object describing the application configuration. Any settings in * this config overrides the default configs as well as system properties. - * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Can - * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] - * from a list of input files or InputFormats for the application. */ -class SparkContext( - config: SparkConf, - // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, - // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It - // contains a map from hostname to a list of input format splits on the host. - val preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) - extends Logging { + +@DeveloperApi +class SparkContext(config: SparkConf) extends Logging { + + // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, + // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It + // contains a map from hostname to a list of input format splits on the host. + private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map() + + /** + * :: DeveloperApi :: + * Alternative constructor for setting preferred locations where Spark will create executors. + * + * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Ca + * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] + * from a list of input files or InputFormats for the application. + */ + @DeveloperApi + def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = { + this(config) + this.preferredNodeLocationData = preferredNodeLocationData + } /** * Alternative constructor that allows setting common Spark properties directly @@ -93,10 +107,45 @@ class SparkContext( environment: Map[String, String] = Map(), preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) = { - this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment), - preferredNodeLocationData) + this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment)) + this.preferredNodeLocationData = preferredNodeLocationData } + // NOTE: The below constructors could be consolidated using default arguments. Due to + // Scala bug SI-8479, however, this causes the compile step to fail when generating docs. + // Until we have a good workaround for that bug the constructors remain broken out. + + /** + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + */ + private[spark] def this(master: String, appName: String) = + this(master, appName, null, Nil, Map(), Map()) + + /** + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + * @param sparkHome Location where Spark is installed on cluster nodes. + */ + private[spark] def this(master: String, appName: String, sparkHome: String) = + this(master, appName, sparkHome, Nil, Map(), Map()) + + /** + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + * @param sparkHome Location where Spark is installed on cluster nodes. + * @param jars Collection of JARs to send to the cluster. These can be paths on the local file + * system or HDFS, HTTP, HTTPS, or FTP URLs. + */ + private[spark] def this(master: String, appName: String, sparkHome: String, jars: Seq[String]) = + this(master, appName, sparkHome, jars, Map(), Map()) + private[spark] val conf = config.clone() /** @@ -189,7 +238,7 @@ class SparkContext( jars.foreach(addJar) } - def warnSparkMem(value: String): String = { + private def warnSparkMem(value: String): String = { logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " + "deprecated, please use spark.executor.memory instead.") value @@ -653,6 +702,9 @@ class SparkContext( def union[T: ClassTag](first: RDD[T], rest: RDD[T]*): RDD[T] = new UnionRDD(this, Seq(first) ++ rest) + /** Get an RDD that has no partitions or elements. */ + def emptyRDD[T: ClassTag] = new EmptyRDD[T](this) + // Methods for creating shared variables /** @@ -716,6 +768,11 @@ class SparkContext( postEnvironmentUpdate() } + /** + * :: DeveloperApi :: + * Register a listener to receive up-calls from events that happen during execution. + */ + @DeveloperApi def addSparkListener(listener: SparkListener) { listenerBus.addListener(listener) } @@ -1021,8 +1078,10 @@ class SparkContext( } /** + * :: DeveloperApi :: * Run a job that can return approximate results. */ + @DeveloperApi def runApproximateJob[T, U, R]( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, @@ -1040,6 +1099,7 @@ class SparkContext( /** * Submit a job for execution and return a FutureJob holding the result. */ + @Experimental def submitJob[T, U, R]( rdd: RDD[T], processPartition: Iterator[T] => U, diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 9ea123f174b95..915315ed74436 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -25,6 +25,7 @@ import scala.util.Properties import akka.actor._ import com.google.common.collect.MapMaker +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem @@ -35,13 +36,18 @@ import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} /** + * :: DeveloperApi :: * Holds all the runtime environment objects for a running Spark instance (either master or worker), * including the serializer, Akka actor system, block manager, map output tracker, etc. Currently * Spark code finds the SparkEnv through a thread-local variable, so each thread that accesses these * objects needs to have the right SparkEnv set. You can get the current environment with * SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set. + * + * NOTE: This is not intended for external use. This is exposed for Shark and may be made private + * in a future release. */ -class SparkEnv private[spark] ( +@DeveloperApi +class SparkEnv ( val executorId: String, val actorSystem: ActorSystem, val serializer: Serializer, diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index be53ca2968cfb..dc5a19ecd738e 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -19,8 +19,14 @@ package org.apache.spark import scala.collection.mutable.ArrayBuffer +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics +/** + * :: DeveloperApi :: + * Contextual information about a task which can be read or mutated during execution. + */ +@DeveloperApi class TaskContext( val stageId: Int, val partitionId: Int, diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index f1a753b6ab8a9..a3074916d13e7 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -17,29 +17,35 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId /** + * :: DeveloperApi :: * Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry * tasks several times for "ephemeral" failures, and only report back failures that require some * old stages to be resubmitted, such as shuffle map fetch failures. */ -private[spark] sealed trait TaskEndReason +@DeveloperApi +sealed trait TaskEndReason -private[spark] case object Success extends TaskEndReason +@DeveloperApi +case object Success extends TaskEndReason -private[spark] +@DeveloperApi case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it -private[spark] case class FetchFailed( +@DeveloperApi +case class FetchFailed( bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int) extends TaskEndReason -private[spark] case class ExceptionFailure( +@DeveloperApi +case class ExceptionFailure( className: String, description: String, stackTrace: Array[StackTraceElement], @@ -47,21 +53,28 @@ private[spark] case class ExceptionFailure( extends TaskEndReason /** + * :: DeveloperApi :: * The task finished successfully, but the result was lost from the executor's block manager before * it was fetched. */ -private[spark] case object TaskResultLost extends TaskEndReason +@DeveloperApi +case object TaskResultLost extends TaskEndReason -private[spark] case object TaskKilled extends TaskEndReason +@DeveloperApi +case object TaskKilled extends TaskEndReason /** + * :: DeveloperApi :: * The task failed because the executor that it was running on was lost. This may happen because * the task crashed the JVM. */ -private[spark] case object ExecutorLostFailure extends TaskEndReason +@DeveloperApi +case object ExecutorLostFailure extends TaskEndReason /** + * :: DeveloperApi :: * We don't know why the task ended -- for example, because of a ClassNotFound exception when * deserializing the task result. */ -private[spark] case object UnknownReason extends TaskEndReason +@DeveloperApi +case object UnknownReason extends TaskEndReason diff --git a/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java b/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java new file mode 100644 index 0000000000000..af01fb7cfbd04 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java @@ -0,0 +1,26 @@ +/* + * 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.annotation; + +import java.lang.annotation.*; + +/** A new component of Spark which may have unstable API's. */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface AlphaComponent {} diff --git a/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java b/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java new file mode 100644 index 0000000000000..5d546e7a63985 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java @@ -0,0 +1,30 @@ +/* + * 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.annotation; + +import java.lang.annotation.*; + +/** + * A lower-level, unstable API intended for developers. + * + * Developer API's might change or be removed in minor versions of Spark. + */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface DeveloperApi {} diff --git a/core/src/main/scala/org/apache/spark/annotation/Experimental.java b/core/src/main/scala/org/apache/spark/annotation/Experimental.java new file mode 100644 index 0000000000000..306b1418d8d0a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/Experimental.java @@ -0,0 +1,31 @@ +/* + * 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.annotation; + +import java.lang.annotation.*; + +/** + * An experimental user-facing API. + * + * Experimental API's might change or be removed in minor versions of Spark, or be adopted as + * first-class Spark API's. + */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface Experimental {} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index f816bb43a5b44..537f410b0ca26 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -23,6 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.Partitioner import org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD @@ -184,14 +185,26 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, Ja def meanApprox(timeout: Long, confidence: JDouble): PartialResult[BoundedDouble] = srdd.meanApprox(timeout, confidence) - /** (Experimental) Approximate operation to return the mean within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the mean within a timeout. + */ + @Experimental def meanApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.meanApprox(timeout) - /** (Experimental) Approximate operation to return the sum within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the sum within a timeout. + */ + @Experimental def sumApprox(timeout: Long, confidence: JDouble): PartialResult[BoundedDouble] = srdd.sumApprox(timeout, confidence) - /** (Experimental) Approximate operation to return the sum within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the sum within a timeout. + */ + @Experimental def sumApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.sumApprox(timeout) /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index e6c5d85917678..a41c7dbda2afc 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -27,11 +27,12 @@ import com.google.common.base.Optional import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{JobConf, OutputFormat} -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, Job} +import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext.rddToPairRDDFunctions +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, PairFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} @@ -201,16 +202,20 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) def countByKey(): java.util.Map[K, Long] = mapAsJavaMap(rdd.countByKey()) /** - * (Experimental) Approximate version of countByKey that can return a partial result if it does + * :: Experimental :: + * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ + @Experimental def countByKeyApprox(timeout: Long): PartialResult[java.util.Map[K, BoundedDouble]] = rdd.countByKeyApprox(timeout).map(mapAsJavaMap) /** - * (Experimental) Approximate version of countByKey that can return a partial result if it does + * :: Experimental :: + * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ + @Experimental def countByKeyApprox(timeout: Long, confidence: Double = 0.95) : PartialResult[java.util.Map[K, BoundedDouble]] = rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap) 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 ae577b500ccb4..725c423a53e35 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 @@ -27,6 +27,7 @@ import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, _} @@ -343,16 +344,20 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def count(): Long = rdd.count() /** - * (Experimental) Approximate version of count() that returns a potentially incomplete result + * :: Experimental :: + * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. */ + @Experimental def countApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = rdd.countApprox(timeout, confidence) /** - * (Experimental) Approximate version of count() that returns a potentially incomplete result + * :: Experimental :: + * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. */ + @Experimental def countApprox(timeout: Long): PartialResult[BoundedDouble] = rdd.countApprox(timeout) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index a2855d4db1d2e..1e8242a2cbbce 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -89,7 +89,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork */ def this(master: String, appName: String, sparkHome: String, jars: Array[String], environment: JMap[String, String]) = - this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment)) + this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment, Map())) private[spark] val env = sc.env diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala index ecbf18849ad48..22810cb1c662d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.api.java import com.google.common.base.Optional -object JavaUtils { +private[spark] object JavaUtils { def optionToOptional[T](option: Option[T]): Optional[T] = option match { case Some(value) => Optional.of(value) diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index c7f7c59cfb449..8c8ce9b1691ac 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -16,16 +16,19 @@ */ package org.apache.spark.broadcast -import org.apache.spark.SecurityManager +import org.apache.spark.SecurityManager import org.apache.spark.SparkConf +import org.apache.spark.annotation.DeveloperApi /** - * An interface for all the broadcast implementations in Spark (to allow + * :: DeveloperApi :: + * An interface for all the broadcast implementations in Spark (to allow * multiple broadcast implementations). SparkContext uses a user-specified * BroadcastFactory implementation to instantiate a particular broadcast for the * entire Spark job. */ +@DeveloperApi trait BroadcastFactory { def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index 127f5e90f3e1a..0ed52cfe9df61 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.fs.FileSystem import org.apache.spark.metrics.source.Source -class ExecutorSource(val executor: Executor, executorId: String) extends Source { +private[spark] class ExecutorSource(val executor: Executor, executorId: String) extends Source { private def fileStats(scheme: String) : Option[FileSystem.Statistics] = FileSystem.getAllStatistics().filter(s => s.getScheme.equals(scheme)).headOption diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 88625e79a5c68..e4f02a4be0b97 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,8 +17,14 @@ package org.apache.spark.executor +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.storage.{BlockId, BlockStatus} +/** + * :: DeveloperApi :: + * Metrics tracked during the execution of a task. + */ +@DeveloperApi class TaskMetrics extends Serializable { /** * Host's name the task runs on @@ -77,11 +83,16 @@ class TaskMetrics extends Serializable { var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None } -object TaskMetrics { - private[spark] def empty(): TaskMetrics = new TaskMetrics +private[spark] object TaskMetrics { + def empty(): TaskMetrics = new TaskMetrics } +/** + * :: DeveloperApi :: + * Metrics pertaining to shuffle data read in a given task. + */ +@DeveloperApi class ShuffleReadMetrics extends Serializable { /** * Absolute time when this task finished reading shuffle data @@ -116,6 +127,11 @@ class ShuffleReadMetrics extends Serializable { var remoteBytesRead: Long = _ } +/** + * :: DeveloperApi :: + * Metrics pertaining to shuffle data written in a given task. + */ +@DeveloperApi class ShuffleWriteMetrics extends Serializable { /** * Number of bytes written for the shuffle by this task diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 059e58824c39b..e1a5ee316bb69 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -23,11 +23,18 @@ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf +import org.apache.spark.annotation.DeveloperApi /** + * :: DeveloperApi :: * CompressionCodec allows the customization of choosing different compression implementations * to be used in block storage. + * + * Note: The wire protocol for a codec is not guaranteed compatible across versions of Spark. + * This is intended for use as an internal compression utility within a single + * Spark application. */ +@DeveloperApi trait CompressionCodec { def compressedOutputStream(s: OutputStream): OutputStream @@ -52,8 +59,14 @@ private[spark] object CompressionCodec { /** + * :: DeveloperApi :: * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. + * + * Note: The wire protocol for this codec is not guaranteed to be compatible across versions + * of Spark. This is intended for use as an internal compression utility within a single Spark + * application. */ +@DeveloperApi class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { @@ -65,9 +78,15 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** + * :: DeveloperApi :: * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. + * + * Note: The wire protocol for this codec is not guaranteed to be compatible across versions + * of Spark. This is intended for use as an internal compression utility within a single Spark + * application. */ +@DeveloperApi class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala index 64eac73605388..05852f1f98993 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala @@ -25,7 +25,7 @@ import com.codahale.metrics.{ConsoleReporter, MetricRegistry} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class ConsoleSink(val property: Properties, val registry: MetricRegistry, +private[spark] class ConsoleSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val CONSOLE_DEFAULT_PERIOD = 10 val CONSOLE_DEFAULT_UNIT = "SECONDS" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala index 544848d4150b6..42c1200926fea 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala @@ -26,7 +26,7 @@ import com.codahale.metrics.{CsvReporter, MetricRegistry} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class CsvSink(val property: Properties, val registry: MetricRegistry, +private[spark] class CsvSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val CSV_KEY_PERIOD = "period" val CSV_KEY_UNIT = "unit" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala index 7f0a2fd16fa99..aeb4ad44a0647 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala @@ -27,7 +27,7 @@ import com.codahale.metrics.graphite.{Graphite, GraphiteReporter} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class GraphiteSink(val property: Properties, val registry: MetricRegistry, +private[spark] class GraphiteSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val GRAPHITE_DEFAULT_PERIOD = 10 val GRAPHITE_DEFAULT_UNIT = "SECONDS" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala index 3b5edd5c376f0..ed27234b4e760 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala @@ -22,7 +22,7 @@ import java.util.Properties import com.codahale.metrics.{JmxReporter, MetricRegistry} import org.apache.spark.SecurityManager -class JmxSink(val property: Properties, val registry: MetricRegistry, +private[spark] class JmxSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val reporter: JmxReporter = JmxReporter.forRegistry(registry).build() diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala index 854b52c510e3d..571539ba5e467 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala @@ -30,7 +30,7 @@ import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.SecurityManager import org.apache.spark.ui.JettyUtils._ -class MetricsServlet(val property: Properties, val registry: MetricRegistry, +private[spark] class MetricsServlet(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val SERVLET_KEY_PATH = "path" val SERVLET_KEY_SAMPLE = "sample" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala index 3a739aa563eae..6f2b5a06027ea 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala @@ -17,7 +17,7 @@ package org.apache.spark.metrics.sink -trait Sink { +private[spark] trait Sink { def start: Unit def stop: Unit } diff --git a/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala index 75cb2b8973aa1..f865f9648a91e 100644 --- a/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala @@ -20,7 +20,7 @@ package org.apache.spark.metrics.source import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} -class JvmSource extends Source { +private[spark] class JvmSource extends Source { val sourceName = "jvm" val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/org/apache/spark/metrics/source/Source.scala b/core/src/main/scala/org/apache/spark/metrics/source/Source.scala index 3fee55cc6dcd5..1dda2cd83b2a9 100644 --- a/core/src/main/scala/org/apache/spark/metrics/source/Source.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/Source.scala @@ -19,7 +19,7 @@ package org.apache.spark.metrics.source import com.codahale.metrics.MetricRegistry -trait Source { +private[spark] trait Source { def sourceName: String def metricRegistry: MetricRegistry } diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 2625a7f6a575a..59bbb1171f239 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -32,7 +32,16 @@ package org.apache * * Java programmers should reference the [[spark.api.java]] package * for Spark programming APIs in Java. + * + * Classes and methods marked with + * Experimental are user-facing features which have not been officially adopted by the + * Spark project. These are subject to change or removal in minor releases. + * + * Classes and methods marked with + * Developer API are intended for advanced users want to extend Spark through lower + * level interfaces. These are subject to changes or removal in minor releases. */ + package object spark { // For package docs only } diff --git a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala index 5f4450859cc9b..aed0353344427 100644 --- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -17,9 +17,13 @@ package org.apache.spark.partial +import org.apache.spark.annotation.Experimental + /** - * A Double with error bars on it. + * :: Experimental :: + * A Double value with error bars and associated confidence. */ +@Experimental class BoundedDouble(val mean: Double, val confidence: Double, val low: Double, val high: Double) { override def toString(): String = "[%.3f, %.3f]".format(low, high) } diff --git a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala index 812368e04ac0d..eade07fbcbe37 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -17,6 +17,9 @@ package org.apache.spark.partial +import org.apache.spark.annotation.Experimental + +@Experimental class PartialResult[R](initialVal: R, isFinal: Boolean) { private var finalValue: Option[R] = if (isFinal) Some(initialVal) else None private var failure: Option[Exception] = None diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index d1c74a5063510..aed951a40b40c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -24,11 +24,14 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} +import org.apache.spark.annotation.Experimental /** + * :: Experimental :: * A set of asynchronous RDD actions available through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ +@Experimental class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Logging { /** 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 9aa454a5c8b88..c6e79557f08a1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -23,6 +23,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.annotation.DeveloperApi import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} import org.apache.spark.serializer.Serializer @@ -51,12 +52,17 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] } /** + * :: DeveloperApi :: * A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a * tuple with the list of values for that key. * + * Note: This is an internal API. We recommend users use RDD.coGroup(...) instead of + * instantiating this directly. + * @param rdds parent RDDs. - * @param part partitioner used to partition the shuffle output. + * @param part partitioner used to partition the shuffle output */ +@DeveloperApi class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index 4e82b51313bf0..44401a663440c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -32,7 +32,7 @@ import org.apache.spark._ * @param parentsIndices list of indices in the parent that have been coalesced into this partition * @param preferredLocation the preferred location for this partition */ -case class CoalescedRDDPartition( +private[spark] case class CoalescedRDDPartition( index: Int, @transient rdd: RDD[_], parentsIndices: Array[Int], @@ -70,7 +70,7 @@ case class CoalescedRDDPartition( * @param maxPartitions number of desired partitions in the coalesced RDD * @param balanceSlack used to trade-off balance and locality. 1.0 is all locality, 0 is all balance */ -class CoalescedRDD[T: ClassTag]( +private[spark] class CoalescedRDD[T: ClassTag]( @transient var prev: RDD[T], maxPartitions: Int, balanceSlack: Double = 0.10) diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index a7b6b3b5146ce..2306c9736b334 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -17,6 +17,7 @@ package org.apache.spark.rdd +import org.apache.spark.annotation.Experimental import org.apache.spark.{TaskContext, Logging} import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.MeanEvaluator @@ -63,14 +64,22 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { */ def sampleVariance(): Double = stats().sampleVariance - /** (Experimental) Approximate operation to return the mean within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the mean within a timeout. + */ + @Experimental def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) val evaluator = new MeanEvaluator(self.partitions.size, confidence) self.context.runApproximateJob(self, processPartition, evaluator, timeout) } - /** (Experimental) Approximate operation to return the sum within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the sum within a timeout. + */ + @Experimental def sumApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) val evaluator = new SumEvaluator(self.partitions.size, confidence) diff --git a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala index a84e5f9fd8ef8..a2d7e344cf1b2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala @@ -22,9 +22,9 @@ import scala.reflect.ClassTag import org.apache.spark.{Partition, SparkContext, TaskContext} /** - * An RDD that is empty, i.e. has no element in it. + * An RDD that has no partitions and no elements. */ -class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { +private[spark] class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { override def getPartitions: Array[Partition] = Array.empty diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 3af008bd72378..6811e1abb8b70 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.mapred.TaskID import org.apache.hadoop.util.ReflectionUtils import org.apache.spark._ +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.NextIterator @@ -70,9 +71,13 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp } /** + * :: DeveloperApi :: * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the older MapReduce API (`org.apache.hadoop.mapred`). * + * Note: Instantiating this class directly is not recommended, please use + * [[org.apache.spark.SparkContext.hadoopRDD()]] + * * @param sc The SparkContext to associate the RDD with. * @param broadcastedConf A general Hadoop Configuration, or a subclass of it. If the enclosed * variabe references an instance of JobConf, then that JobConf will be used for the Hadoop job. @@ -84,6 +89,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp * @param valueClass Class of the value associated with the inputFormatClass. * @param minSplits Minimum number of Hadoop Splits (HadoopRDD partitions) to generate. */ +@DeveloperApi class HadoopRDD[K, V]( sc: SparkContext, broadcastedConf: Broadcast[SerializableWritable[Configuration]], diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 1b503743ac117..a76a070b5b863 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.NextIterator private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { override def index = idx } - +// TODO: Expose a jdbcRDD function in SparkContext and mark this as semi-private /** * An RDD that executes an SQL query on a JDBC connection and reads results. * For usage example, see test case JdbcRDDSuite. diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 461a749eac48b..2d8dfa5a1645a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ import org.apache.spark.{InterruptibleIterator, Logging, Partition, SerializableWritable, SparkContext, TaskContext} +import org.apache.spark.annotation.DeveloperApi private[spark] class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable) @@ -36,15 +37,20 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS } /** + * :: DeveloperApi :: * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the new MapReduce API (`org.apache.hadoop.mapreduce`). * + * Note: Instantiating this class directly is not recommended, please use + * [[org.apache.spark.SparkContext.newAPIHadoopRDD()]] + * * @param sc The SparkContext to associate the RDD with. * @param inputFormatClass Storage format of the data to be read. * @param keyClass Class of the key associated with the inputFormatClass. * @param valueClass Class of the value associated with the inputFormatClass. * @param conf The Hadoop configuration. */ +@DeveloperApi class NewHadoopRDD[K, V]( sc : SparkContext, inputFormatClass: Class[_ <: InputFormat[K, V]], 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 a92a84b5342d1..343e4325c0ef0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -39,6 +39,7 @@ RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} import org.apache.spark._ +import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.SparkHadoopWriter import org.apache.spark.Partitioner.defaultPartitioner @@ -201,9 +202,11 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def countByKey(): Map[K, Long] = self.map(_._1).countByValue() /** - * (Experimental) Approximate version of countByKey that can return a partial result if it does + * :: Experimental :: + * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ + @Experimental def countByKeyApprox(timeout: Long, confidence: Double = 0.95) : PartialResult[Map[K, BoundedDouble]] = { self.map(_._1).countByValueApprox(timeout, confidence) diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala index b0440ca7f32cf..f781a8d776f2a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -20,8 +20,10 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{NarrowDependency, Partition, TaskContext} +import org.apache.spark.annotation.DeveloperApi -class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition { +private[spark] class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) + extends Partition { override val index = idx } @@ -30,7 +32,7 @@ class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends * Represents a dependency between the PartitionPruningRDD and its parent. In this * case, the child RDD contains a subset of partitions of the parents'. */ -class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boolean) +private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boolean) extends NarrowDependency[T](rdd) { @transient @@ -45,11 +47,13 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo /** + * :: DeveloperApi :: * A RDD used to prune RDD partitions/partitions so we can avoid launching tasks on * all partitions. An example use case: If we know the RDD is partitioned by range, * and the execution DAG has a filter on the key, we can avoid launching tasks * on partitions that don't have the range covering the key. */ +@DeveloperApi class PartitionPruningRDD[T: ClassTag]( @transient prev: RDD[T], @transient partitionFilterFunc: Int => Boolean) @@ -63,6 +67,7 @@ class PartitionPruningRDD[T: ClassTag]( } +@DeveloperApi object PartitionPruningRDD { /** diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala index ce4c0d382baab..b4e3bb5d75e17 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala @@ -42,7 +42,7 @@ class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) * @tparam T input RDD item type * @tparam U sampled RDD item type */ -class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( +private[spark] class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( prev: RDD[T], sampler: RandomSampler[T, U], @transient seed: Long = System.nanoTime) diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 41ae0fec823e7..e441d4a40ccd2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -37,7 +37,7 @@ import org.apache.spark.util.Utils * An RDD that pipes the contents of each parent partition through an external command * (printing them one per line) and returns the output as a collection of strings. */ -class PipedRDD[T: ClassTag]( +private[spark] class PipedRDD[T: ClassTag]( prev: RDD[T], command: Seq[String], envVars: Map[String, String], 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 74fa2a4fcd401..3437b2cac19c2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.mapred.TextOutputFormat import org.apache.spark._ import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator @@ -86,22 +87,34 @@ abstract class RDD[T: ClassTag]( // Methods that should be implemented by subclasses of RDD // ======================================================================= - /** Implemented by subclasses to compute a given partition. */ + /** + * :: DeveloperApi :: + * Implemented by subclasses to compute a given partition. + */ + @DeveloperApi def compute(split: Partition, context: TaskContext): Iterator[T] /** + * :: DeveloperApi :: * Implemented by subclasses to return the set of partitions in this RDD. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ + @DeveloperApi protected def getPartitions: Array[Partition] /** + * :: DeveloperApi :: * Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ + @DeveloperApi protected def getDependencies: Seq[Dependency[_]] = deps - /** Optionally overridden by subclasses to specify placement preferences. */ + /** + * :: DeveloperApi :: + * Optionally overridden by subclasses to specify placement preferences. + */ + @DeveloperApi protected def getPreferredLocations(split: Partition): Seq[String] = Nil /** Optionally overridden by subclasses to specify how they are partitioned. */ @@ -518,9 +531,11 @@ abstract class RDD[T: ClassTag]( } /** + * :: DeveloperApi :: * Return a new RDD by applying a function to each partition of this RDD. This is a variant of * mapPartitions that also passes the TaskContext into the closure. */ + @DeveloperApi def mapPartitionsWithContext[U: ClassTag]( f: (TaskContext, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { @@ -792,9 +807,11 @@ abstract class RDD[T: ClassTag]( def count(): Long = sc.runJob(this, Utils.getIteratorSize _).sum /** - * (Experimental) Approximate version of count() that returns a potentially incomplete result + * :: Experimental :: + * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. */ + @Experimental def countApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val countElements: (TaskContext, Iterator[T]) => Long = { (ctx, iter) => var result = 0L @@ -838,8 +855,10 @@ abstract class RDD[T: ClassTag]( } /** - * (Experimental) Approximate version of countByValue(). + * :: Experimental :: + * Approximate version of countByValue(). */ + @Experimental def countByValueApprox( timeout: Long, confidence: Double = 0.95 @@ -860,6 +879,7 @@ abstract class RDD[T: ClassTag]( } /** + * :: Experimental :: * Return approximate number of distinct elements in the RDD. * * The accuracy of approximation can be controlled through the relative standard deviation @@ -867,6 +887,7 @@ abstract class RDD[T: ClassTag]( * more accurate counts but increase the memory footprint and vise versa. The default value of * relativeSD is 0.05. */ + @Experimental def countApproxDistinct(relativeSD: Double = 0.05): Long = { val zeroCounter = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) aggregate(zeroCounter)(_.add(_), _.merge(_)).value.cardinality() 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 4ceea557f569c..b097c30f8c231 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala @@ -33,7 +33,7 @@ class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition } @deprecated("Replaced by PartitionwiseSampledRDD", "1.0.0") -class SampledRDD[T: ClassTag]( +private[spark] class SampledRDD[T: ClassTag]( prev: RDD[T], withReplacement: Boolean, frac: Double, 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 02660ea6a45c5..802b0bdfb2d59 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.annotation.DeveloperApi import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -28,12 +29,14 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { } /** + * :: DeveloperApi :: * The resulting RDD from a shuffle (e.g. repartitioning of data). * @param prev the parent RDD. * @param part the partitioner used to partition the RDD * @tparam K the key class. * @tparam V the value class. */ +@DeveloperApi class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( @transient var prev: RDD[P], part: Partitioner) diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index a447030752096..21c6e07d69f90 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} +import org.apache.spark.annotation.DeveloperApi private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitIndex: Int) extends Partition { @@ -43,6 +44,7 @@ private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitInd } } +@DeveloperApi class UnionRDD[T: ClassTag]( sc: SparkContext, @transient var rdds: Seq[RDD[T]]) diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index b56643444aa40..f3d30f6c9b32f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -41,7 +41,7 @@ private[spark] class ZippedPartitionsPartition( } } -abstract class ZippedPartitionsBaseRDD[V: ClassTag]( +private[spark] abstract class ZippedPartitionsBaseRDD[V: ClassTag]( sc: SparkContext, var rdds: Seq[RDD[_]], preservesPartitioning: Boolean = false) @@ -74,7 +74,7 @@ abstract class ZippedPartitionsBaseRDD[V: ClassTag]( } } -class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( +private[spark] class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B]) => Iterator[V], var rdd1: RDD[A], @@ -94,7 +94,7 @@ class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( } } -class ZippedPartitionsRDD3 +private[spark] class ZippedPartitionsRDD3 [A: ClassTag, B: ClassTag, C: ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V], @@ -119,7 +119,7 @@ class ZippedPartitionsRDD3 } } -class ZippedPartitionsRDD4 +private[spark] class ZippedPartitionsRDD4 [A: ClassTag, B: ClassTag, C: ClassTag, D:ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala index 2119e76f0e032..b8110ffc42f2d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala @@ -44,7 +44,7 @@ private[spark] class ZippedPartition[T: ClassTag, U: ClassTag]( } } -class ZippedRDD[T: ClassTag, U: ClassTag]( +private[spark] class ZippedRDD[T: ClassTag, U: ClassTag]( sc: SparkContext, var rdd1: RDD[T], var rdd2: RDD[U]) diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index b3f2cb346f7da..bac37bfdaa23f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -27,11 +27,14 @@ import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.Logging +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil /** + * :: DeveloperApi :: * Parses and holds information about inputFormat (and files) specified as a parameter. */ +@DeveloperApi class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], val path: String) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 7c5053998f1d6..713aebfa3ce00 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -25,9 +25,11 @@ import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.HashMap import org.apache.spark._ +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics /** + * :: DeveloperApi :: * A logger class to record runtime information for jobs in Spark. This class outputs one log file * for each Spark job, containing tasks start/stop and shuffle information. JobLogger is a subclass * of SparkListener, use addSparkListener to add JobLogger to a SparkContext after the SparkContext @@ -38,7 +40,7 @@ import org.apache.spark.executor.TaskMetrics * to log application information as SparkListenerEvents. To enable this functionality, set * spark.eventLog.enabled to true. */ - +@DeveloperApi @deprecated("Log application information by setting spark.eventLog.enabled.", "1.0.0") class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index 047bd27056120..4cd6cbe189aab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -17,11 +17,17 @@ package org.apache.spark.scheduler +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A result of a job in the DAGScheduler. */ -private[spark] sealed trait JobResult +@DeveloperApi +sealed trait JobResult -private[spark] case object JobSucceeded extends JobResult +@DeveloperApi +case object JobSucceeded extends JobResult +@DeveloperApi private[spark] case class JobFailed(exception: Exception) extends JobResult diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index d42e67742a4f7..ced20350d5356 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -23,21 +23,28 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.{Logging, TaskEndReason} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} +@DeveloperApi sealed trait SparkListenerEvent +@DeveloperApi case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) extends SparkListenerEvent +@DeveloperApi case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent +@DeveloperApi case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent +@DeveloperApi case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent +@DeveloperApi case class SparkListenerTaskEnd( stageId: Int, taskType: String, @@ -46,20 +53,26 @@ case class SparkListenerTaskEnd( taskMetrics: TaskMetrics) extends SparkListenerEvent +@DeveloperApi case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) extends SparkListenerEvent +@DeveloperApi case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent +@DeveloperApi case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent +@DeveloperApi case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent +@DeveloperApi case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) extends SparkListenerEvent +@DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ @@ -67,8 +80,11 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** - * Interface for listening to events from the Spark scheduler. + * :: DeveloperApi :: + * Interface for listening to events from the Spark scheduler. Note that this is an internal + * interface which might change in different Spark releases. */ +@DeveloperApi trait SparkListener { /** * Called when a stage completes successfully or fails, with information on the completed stage. @@ -128,8 +144,10 @@ trait SparkListener { } /** + * :: DeveloperApi :: * Simple SparkListener that logs a few summary statistics when each stage completes */ +@DeveloperApi class StatsReportListener extends SparkListener with Logging { import org.apache.spark.scheduler.StatsReportListener._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala index 5b40a3eb29b30..b85eabd6bbdbc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -19,8 +19,11 @@ package org.apache.spark.scheduler import collection.mutable.ArrayBuffer +import org.apache.spark.annotation.DeveloperApi + // information about a specific split instance : handles both split instances. // So that we do not need to worry about the differences. +@DeveloperApi class SplitInfo(val inputFormatClazz: Class[_], val hostLocation: String, val path: String, val length: Long, val underlyingSplit: Any) { override def toString(): String = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index eec409b182ac6..9f732f7191465 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -17,12 +17,14 @@ package org.apache.spark.scheduler +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.storage.RDDInfo /** + * :: DeveloperApi :: * Stores information about a stage to pass from the scheduler to SparkListeners. */ -private[spark] +@DeveloperApi class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 6183b125def99..4c62e4dc0bac8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -17,10 +17,13 @@ package org.apache.spark.scheduler +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * Information about a running task attempt inside a TaskSet. */ -private[spark] +@DeveloperApi class TaskInfo( val taskId: Long, val index: Int, @@ -46,15 +49,15 @@ class TaskInfo( var serializedSize: Int = 0 - def markGettingResult(time: Long = System.currentTimeMillis) { + private[spark] def markGettingResult(time: Long = System.currentTimeMillis) { gettingResultTime = time } - def markSuccessful(time: Long = System.currentTimeMillis) { + private[spark] def markSuccessful(time: Long = System.currentTimeMillis) { finishTime = time } - def markFailed(time: Long = System.currentTimeMillis) { + private[spark] def markFailed(time: Long = System.currentTimeMillis) { finishTime = time failed = true } @@ -83,11 +86,11 @@ class TaskInfo( def duration: Long = { if (!finished) { - throw new UnsupportedOperationException("duration() called on unfinished tasks") + throw new UnsupportedOperationException("duration() called on unfinished task") } else { finishTime - launchTime } } - def timeRunning(currentTime: Long): Long = currentTime - launchTime + private[spark] def timeRunning(currentTime: Long): Long = currentTime - launchTime } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index 308edb12edd5c..eb920ab0c0b67 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -17,7 +17,10 @@ package org.apache.spark.scheduler -private[spark] object TaskLocality extends Enumeration { +import org.apache.spark.annotation.DeveloperApi + +@DeveloperApi +object TaskLocality extends Enumeration { // Process local is expected to be used ONLY within TaskSetManager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value 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 18a68b05fa853..5e5883554fcc1 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -21,6 +21,7 @@ import java.io._ import java.nio.ByteBuffer import org.apache.spark.SparkConf +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.ByteBufferInputStream private[spark] class JavaSerializationStream(out: OutputStream, counterReset: Int) @@ -94,8 +95,14 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } /** + * :: DeveloperApi :: * A Spark serializer that uses Java's built-in serialization. + * + * Note that this serializer is not guaranteed to be wire-compatible across different versions of + * Spark. It is intended to be used to serialize/de-serialize data within a single + * Spark application. */ +@DeveloperApi class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 10000) 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 926e71573be32..d1e8c3ef63622 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -33,6 +33,10 @@ import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. + * + * Note that this serializer is not guaranteed to be wire-compatible across different versions of + * Spark. It is intended to be used to serialize/de-serialize data within a single + * Spark application. */ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer 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 099143494b851..9f04dc6e427c0 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -22,20 +22,27 @@ import java.nio.ByteBuffer import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import org.apache.spark.util.{ByteBufferInputStream, NextIterator} import org.apache.spark.SparkEnv +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.{ByteBufferInputStream, NextIterator} /** + * :: DeveloperApi :: * 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 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. + * + * Note that serializers are not required to be wire-compatible across different versions of Spark. + * They are intended to be used to serialize/de-serialize data within a single Spark application. */ +@DeveloperApi trait Serializer { def newInstance(): SerializerInstance } @@ -49,8 +56,10 @@ object Serializer { /** + * :: DeveloperApi :: * An instance of a serializer, for use by one thread at a time. */ +@DeveloperApi trait SerializerInstance { def serialize[T](t: T): ByteBuffer @@ -81,8 +90,10 @@ trait SerializerInstance { /** + * :: DeveloperApi :: * A stream for writing serialized objects. */ +@DeveloperApi trait SerializationStream { def writeObject[T](t: T): SerializationStream def flush(): Unit @@ -98,8 +109,10 @@ trait SerializationStream { /** + * :: DeveloperApi :: * A stream for reading serialized objects. */ +@DeveloperApi trait DeserializationStream { def readObject[T](): T def close(): Unit diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index b021564477c47..a2a729130091f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -35,11 +35,11 @@ import org.apache.spark.network._ import org.apache.spark.serializer.Serializer import org.apache.spark.util._ -sealed trait Values +private[spark] sealed trait Values -case class ByteBufferValues(buffer: ByteBuffer) extends Values -case class IteratorValues(iterator: Iterator[Any]) extends Values -case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends Values +private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends Values +private[spark] case class IteratorValues(iterator: Iterator[Any]) extends Values +private[spark] case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends Values private[spark] class BlockManager( executorId: String, diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index ff6e84cf9819a..07255aa366a6d 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -21,6 +21,7 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.SparkContext +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils private[spark] @@ -47,6 +48,7 @@ class StorageStatus( } } +@DeveloperApi private[spark] class RDDInfo( val id: Int, diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index c3692f2fd929b..b9f4a5d720b93 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -28,7 +28,7 @@ import scala.collection.generic.Growable * class and modifies it such that only the top K elements are retained. * The top K elements are defined by an implicit Ordering[A]. */ -class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) +private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) extends Iterable[A] with Growable[A] with Serializable { private val underlying = new JPriorityQueue[A](maxSize, ord) diff --git a/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala b/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala index db3db87e6618e..93235031f3ad5 100644 --- a/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala +++ b/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala @@ -22,7 +22,7 @@ import java.util import scala.Array import scala.reflect._ -object CollectionsUtils { +private[spark] object CollectionsUtils { def makeBinarySearch[K <% Ordered[K] : ClassTag] : (Array[K], K) => Int = { classTag[K] match { case ClassTag.Float => diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala index 5b347555fe708..a465298c8c5ab 100644 --- a/core/src/main/scala/org/apache/spark/util/Distribution.scala +++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala @@ -29,7 +29,7 @@ import scala.collection.immutable.IndexedSeq * * Assumes you are giving it a non-empty set of data */ -class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) { +private[spark] class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) { require(startIdx < endIdx) def this(data: Traversable[Double]) = this(data.toArray, 0, data.size) java.util.Arrays.sort(data, startIdx, endIdx) @@ -69,7 +69,7 @@ class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) } } -object Distribution { +private[spark] object Distribution { def apply(data: Traversable[Double]): Option[Distribution] = { if (data.size > 0) { diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index a0c07e32fdc98..b5f2ec6831d26 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -36,7 +36,7 @@ import org.apache.spark.io.CompressionCodec * @param compress Whether to compress output * @param overwrite Whether to overwrite existing files */ -class FileLogger( +private[spark] class FileLogger( logDir: String, conf: SparkConf = new SparkConf, outputBufferSize: Int = 8 * 1024, // 8 KB 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 a6b39247a54ca..74fa77b68de0b 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -17,13 +17,17 @@ package org.apache.spark.util +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A tuple of 2 elements. This can be used as an alternative to Scala's Tuple2 when we want to * minimize object allocation. * * @param _1 Element 1 of this MutablePair * @param _2 Element 2 of this MutablePair */ +@DeveloperApi case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T1, @specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T2] (var _1: T1, var _2: T2) diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala index 19bece86b36b4..7cd8f28b12dd6 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala @@ -22,7 +22,7 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions import scala.collection.mutable.Set -class TimeStampedHashSet[A] extends Set[A] { +private[spark] class TimeStampedHashSet[A] extends Set[A] { val internalMap = new ConcurrentHashMap[A, Long]() def contains(key: A): Boolean = { diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index dc4b8f253f259..3c8f94a416c65 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -21,6 +21,7 @@ import scala.util.Random import org.apache.spark.util.random.XORShiftRandom +@deprecated("Use Vectors.dense from Spark's mllib.linalg package instead.", "1.0.0") class Vector(val elements: Array[Double]) extends Serializable { def length = elements.length diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index b8c852b4ff5c7..025492b177a77 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -19,7 +19,10 @@ package org.apache.spark.util.collection import java.util.{Arrays, Comparator} +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A simple open hash table optimized for the append-only use case, where keys * are never removed, but the value for each key may be changed. * @@ -29,9 +32,9 @@ import java.util.{Arrays, Comparator} * * TODO: Cache the hash values of each key? java.util.HashMap does that. */ -private[spark] -class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, - V)] with Serializable { +@DeveloperApi +class AppendOnlyMap[K, V](initialCapacity: Int = 64) + extends Iterable[(K, V)] with Serializable { require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") require(initialCapacity >= 1, "Invalid initial capacity") 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 caa06d5b445b4..dd01ae821f705 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 @@ -27,10 +27,12 @@ import com.google.common.io.ByteStreams import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, BlockManager} /** + * :: DeveloperApi :: * An append-only map that spills sorted content to disk when there is insufficient space for it * to grow. * @@ -55,8 +57,8 @@ import org.apache.spark.storage.{BlockId, BlockManager} * `spark.shuffle.safetyFraction` specifies an additional margin of safety as a fraction of * this threshold, in case map size estimation is not sufficiently accurate. */ - -private[spark] class ExternalAppendOnlyMap[K, V, C]( +@DeveloperApi +class ExternalAppendOnlyMap[K, V, C]( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index c26f23d50024a..62f99f3981793 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -19,14 +19,17 @@ package org.apache.spark.util.collection import scala.reflect.ClassTag +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A fast hash map implementation for nullable keys. This hash map supports insertions and updates, * but not deletions. This map is about 5X faster than java.util.HashMap, while using much less * space overhead. * * Under the hood, it uses our OpenHashSet implementation. */ -private[spark] +@DeveloperApi class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( initialCapacity: Int) extends Iterable[(K, V)] diff --git a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala index 98569143ee1e3..70f3dd62b9b19 100644 --- a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala @@ -17,9 +17,13 @@ package org.apache.spark.util.random +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A class with pseudorandom behavior. */ +@DeveloperApi trait Pseudorandom { /** Set random seed. */ def setSeed(seed: Long) diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index 0f1fca4813ba9..37a6b04f5200f 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -22,7 +22,10 @@ import java.util.Random import cern.jet.random.Poisson import cern.jet.random.engine.DRand +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A pseudorandom sampler. It is possible to change the sampled item type. For example, we might * want to add weights for stratified sampling or importance sampling. Should only use * transformations that are tied to the sampler and cannot be applied after sampling. @@ -30,6 +33,7 @@ import cern.jet.random.engine.DRand * @tparam T item type * @tparam U sampled item type */ +@DeveloperApi trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable { /** take a random sample */ @@ -40,6 +44,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable } /** + * :: DeveloperApi :: * A sampler based on Bernoulli trials. * * @param lb lower bound of the acceptance range @@ -47,6 +52,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable * @param complement whether to use the complement of the range specified, default to false * @tparam T item type */ +@DeveloperApi class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) (implicit random: Random = new XORShiftRandom) extends RandomSampler[T, T] { @@ -67,11 +73,13 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) } /** + * :: DeveloperApi :: * A sampler based on values drawn from Poisson distribution. * * @param poisson a Poisson random number generator * @tparam T item type */ +@DeveloperApi class PoissonSampler[T](mean: Double) (implicit var poisson: Poisson = new Poisson(mean, new DRand)) extends RandomSampler[T, T] { diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 2245bcbc70f1e..bbd56d2fd13bb 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -24,7 +24,9 @@ external_projects = ["flume", "kafka", "mqtt", "twitter", "zeromq"] sql_projects = ["catalyst", "core", "hive"] - projects = core_projects + external_projects.map { |project_name| "external/" + project_name } + projects = core_projects + projects = projects + external_projects.map { |project_name| "external/" + project_name } + projects = projects + sql_projects.map { |project_name| "sql/" + project_name } puts "Moving to project root and building scaladoc." curr_dir = pwd @@ -42,24 +44,22 @@ source = "../" + project_name + "/target/scala-2.10/api" dest = "api/" + project_name - puts "echo making directory " + dest + puts "making directory " + dest mkdir_p dest # From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't. puts "cp -r " + source + "/. " + dest cp_r(source + "/.", dest) - end - - sql_projects.each do |project_name| - source = "../sql/" + project_name + "/target/scala-2.10/api/" - dest = "api/sql/" + project_name - puts "echo making directory " + dest - mkdir_p dest + # Append custom JavaScript + js = File.readlines("./js/api-docs.js") + js_file = dest + "/lib/template.js" + File.open(js_file, 'a') { |f| f.write("\n" + js.join()) } - # From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't. - puts "cp -r " + source + "/. " + dest - cp_r(source + "/.", dest) + # Append custom CSS + css = File.readlines("./css/api-docs.css") + css_file = dest + "/lib/template.css" + File.open(css_file, 'a') { |f| f.write("\n" + css.join()) } end # Build Epydoc for Python diff --git a/docs/css/api-docs.css b/docs/css/api-docs.css new file mode 100644 index 0000000000000..b2d1d7f869790 --- /dev/null +++ b/docs/css/api-docs.css @@ -0,0 +1,18 @@ +/* Dynamically injected style for the API docs */ + +.developer { + background-color: #44751E; +} + +.experimental { + background-color: #257080; +} + +.alphaComponent { + background-color: #bb0000; +} + +.badge { + font-family: Arial, san-serif; + float: right; +} diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js new file mode 100644 index 0000000000000..1414b6d0b81a1 --- /dev/null +++ b/docs/js/api-docs.js @@ -0,0 +1,26 @@ +/* Dynamically injected post-processing code for the API docs */ + +$(document).ready(function() { + var annotations = $("dt:contains('Annotations')").next("dd").children("span.name"); + addBadges(annotations, "AlphaComponent", ":: AlphaComponent ::", "Alpha Component"); + addBadges(annotations, "DeveloperApi", ":: DeveloperApi ::", "Developer API"); + addBadges(annotations, "Experimental", ":: Experimental ::", "Experimental"); +}); + +function addBadges(allAnnotations, name, tag, html) { + var annotations = allAnnotations.filter(":contains('" + name + "')") + var tags = $(".cmt:contains(" + tag + ")") + + // Remove identifier tags from comments + tags.each(function(index) { + var oldHTML = $(this).html(); + var newHTML = oldHTML.replace(tag, ""); + $(this).html(newHTML); + }); + + // Add badges to all containers + tags.prevAll("h4.signature") + .add(annotations.closest("div.fullcommenttop")) + .add(annotations.closest("div.fullcomment").prevAll("h4.signature")) + .prepend(html); +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala index 425a5164cad24..ff17edeaf8f16 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -19,7 +19,10 @@ package org.apache.spark import org.apache.spark.util.collection.OpenHashSet -/** GraphX is a graph processing framework built on top of Spark. */ +/** + * ALPHA COMPONENT + * GraphX is a graph processing framework built on top of Spark. + */ package object graphx { /** * A 64-bit vertex identifier that uniquely identifies a vertex within a graph. It does not need diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 0f06ea088e1a1..77b5429bad432 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -90,7 +90,7 @@ trait MutableRow extends Row { def setString(ordinal: Int, value: String) /** - * EXPERIMENTAL + * Experimental * * Returns a mutable string builder for the specified column. A given row should return the * result of any mutations made to the returned buffer next time getString is called for the same diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 3193787680d16..d3d4c56bafe41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -21,6 +21,7 @@ import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag import org.apache.spark.SparkContext +import org.apache.spark.annotation.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl @@ -32,14 +33,14 @@ import org.apache.spark.sql.columnar.InMemoryColumnarTableScan import org.apache.spark.sql.execution._ /** - * ALPHA COMPONENT - * + * :: AlphaComponent :: * The entry point for running relational queries using Spark. Allows the creation of [[SchemaRDD]] * objects and the execution of SQL queries. * * @groupname userf Spark SQL Functions * @groupname Ungrouped Support functions for language integrated queries. */ +@AlphaComponent class SQLContext(@transient val sparkContext: SparkContext) extends Logging with dsl.ExpressionConversions @@ -63,12 +64,12 @@ class SQLContext(@transient val sparkContext: SparkContext) new this.QueryExecution { val logical = plan } /** - * EXPERIMENTAL - * + * :: Experimental :: * Allows catalyst LogicalPlans to be executed as a SchemaRDD. Note that the LogicalPlan * interface is considered internal, and thus not guranteed to be stable. As a result, using * them directly is not reccomended. */ + @Experimental implicit def logicalPlanToSparkQuery(plan: LogicalPlan): SchemaRDD = new SchemaRDD(this, plan) /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index fc95781448569..16da7fd92bffe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} +import org.apache.spark.annotation.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ @@ -26,8 +27,7 @@ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.types.BooleanType /** - * ALPHA COMPONENT - * + * :: AlphaComponent :: * An RDD of [[Row]] objects that has an associated schema. In addition to standard RDD functions, * SchemaRDDs can be used in relational queries, as shown in the examples below. * @@ -90,6 +90,7 @@ import org.apache.spark.sql.catalyst.types.BooleanType * @groupprio schema -1 * @groupname Ungrouped Base RDD Functions */ +@AlphaComponent class SchemaRDD( @transient val sqlContext: SQLContext, @transient protected[spark] val logicalPlan: LogicalPlan) @@ -228,8 +229,7 @@ class SchemaRDD( Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan)) /** - * EXPERIMENTAL - * + * :: Experimental :: * Filters tuples using a function over a `Dynamic` version of a given Row. DynamicRows use * scala's Dynamic trait to emulate an ORM of in a dynamically typed language. Since the type of * the column is not known at compile time, all attributes are converted to strings before @@ -241,18 +241,19 @@ class SchemaRDD( * * @group Query */ + @Experimental def where(dynamicUdf: (DynamicRow) => Boolean) = new SchemaRDD( sqlContext, Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(logicalPlan.output))), logicalPlan)) /** - * EXPERIMENTAL - * + * :: Experimental :: * Returns a sampled version of the underlying dataset. * * @group Query */ + @Experimental def sample( fraction: Double, withReplacement: Boolean = true, @@ -260,8 +261,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan)) /** - * EXPERIMENTAL - * + * :: Experimental :: * Applies the given Generator, or table generating function, to this relation. * * @param generator A table generating function. The API for such functions is likely to change @@ -277,6 +277,7 @@ class SchemaRDD( * * @group Query */ + @Experimental def generate( generator: Generator, join: Boolean = false, @@ -285,8 +286,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Generate(generator, join, outer, None, logicalPlan)) /** - * EXPERIMENTAL - * + * :: Experimental :: * Adds the rows from this RDD to the specified table. Note in a standard [[SQLContext]] there is * no notion of persistent tables, and thus queries that contain this operator will fail to * optimize. When working with an extension of a SQLContext that has a persistent catalog, such @@ -294,6 +294,7 @@ class SchemaRDD( * * @group schema */ + @Experimental def insertInto(tableName: String, overwrite: Boolean = false) = new SchemaRDD( sqlContext, diff --git a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala rename to tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index 8cea302eb14c3..8e8c35615a711 100644 --- a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.storage +package org.apache.spark.tools import java.util.concurrent.{CountDownLatch, Executors} import java.util.concurrent.atomic.AtomicLong @@ -25,7 +25,7 @@ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.Utils /** - * Utility for micro-benchmarking shuffle write performance. + * Internal utility for micro-benchmarking shuffle write performance. * * Writes simulated shuffle output from several threads and records the observed throughput. */ From bde9cc11fee42a0a41ec52d5dc7fa0502ce94f77 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 9 Apr 2014 02:21:15 -0700 Subject: [PATCH 037/641] [SPARK-1357] [MLLIB] Annotate developer and experimental APIs Annotate developer and experimental APIs in MLlib. Author: Xiangrui Meng Closes #298 from mengxr/api and squashes the following commits: 13390e8 [Xiangrui Meng] Merge branch 'master' into api dc4cbb3 [Xiangrui Meng] mark distribute matrices experimental 6b9f8e2 [Xiangrui Meng] add Experimental annotation 8773d0d [Xiangrui Meng] add DeveloperApi annotation da31733 [Xiangrui Meng] update developer and experimental tags 555e0fe [Xiangrui Meng] Merge branch 'master' into api ef1a717 [Xiangrui Meng] mark some constructors private add default parameters to JavaDoc 00ffbcc [Xiangrui Meng] update tree API annotation 0b674fa [Xiangrui Meng] mark decision tree APIs 86b9e34 [Xiangrui Meng] one pass over APIs of GLMs, NaiveBayes, and ALS f21d862 [Xiangrui Meng] Merge branch 'master' into api 2b133d6 [Xiangrui Meng] intial annotation of developer and experimental apis --- .../mllib/api/python/PythonMLLibAPI.scala | 4 ++ .../classification/LogisticRegression.scala | 18 +++---- .../mllib/classification/NaiveBayes.scala | 29 +++++++---- .../spark/mllib/classification/SVM.scala | 21 ++++---- .../spark/mllib/clustering/KMeans.scala | 49 ++++++++++++++++--- .../apache/spark/mllib/linalg/Vectors.scala | 19 ++++--- .../linalg/distributed/CoordinateMatrix.scala | 4 ++ .../distributed/DistributedMatrix.scala | 2 - .../linalg/distributed/IndexedRowMatrix.scala | 11 ++++- .../mllib/linalg/distributed/RowMatrix.scala | 4 ++ .../spark/mllib/optimization/Gradient.scala | 13 +++++ .../mllib/optimization/GradientDescent.scala | 15 ++++-- .../spark/mllib/optimization/Optimizer.scala | 7 +++ .../spark/mllib/optimization/Updater.scala | 13 +++++ .../spark/mllib/recommendation/ALS.scala | 34 ++++++++----- .../MatrixFactorizationModel.scala | 9 ++-- .../GeneralizedLinearAlgorithm.scala | 7 ++- .../apache/spark/mllib/regression/Lasso.scala | 18 ++++--- .../mllib/regression/LinearRegression.scala | 16 +++--- .../mllib/regression/RidgeRegression.scala | 20 ++++---- .../spark/mllib/tree/DecisionTree.scala | 10 ++-- .../spark/mllib/tree/configuration/Algo.scala | 5 ++ .../tree/configuration/FeatureType.scala | 5 ++ .../tree/configuration/QuantileStrategy.scala | 5 ++ .../mllib/tree/configuration/Strategy.scala | 6 ++- .../spark/mllib/tree/impurity/Entropy.scala | 32 +++++++----- .../spark/mllib/tree/impurity/Gini.scala | 10 +++- .../spark/mllib/tree/impurity/Impurity.scala | 12 ++++- .../spark/mllib/tree/impurity/Variance.scala | 8 +++ .../apache/spark/mllib/tree/model/Bin.scala | 1 + .../mllib/tree/model/DecisionTreeModel.scala | 4 ++ .../spark/mllib/tree/model/Filter.scala | 2 +- .../tree/model/InformationGainStats.scala | 5 ++ .../apache/spark/mllib/tree/model/Node.scala | 4 ++ .../apache/spark/mllib/tree/model/Split.scala | 12 +++-- .../spark/mllib/util/DataValidators.scala | 8 +-- .../mllib/util/KMeansDataGenerator.scala | 5 +- .../mllib/util/LinearDataGenerator.scala | 4 ++ .../LogisticRegressionDataGenerator.scala | 5 +- .../spark/mllib/util/MFDataGenerator.scala | 8 +-- .../org/apache/spark/mllib/util/MLUtils.scala | 9 +++- .../spark/mllib/util/SVMDataGenerator.scala | 4 ++ 42 files changed, 355 insertions(+), 122 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 2df5b0d02b699..ae27c57799873 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.api.python import java.nio.{ByteBuffer, ByteOrder} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ @@ -28,8 +29,11 @@ import org.apache.spark.mllib.regression._ import org.apache.spark.rdd.RDD /** + * :: DeveloperApi :: + * * The Java stubs necessary for the Python mllib bindings. */ +@DeveloperApi class PythonMLLibAPI extends Serializable { private def deserializeDoubleVector(bytes: Array[Byte]): Array[Double] = { val packetLength = bytes.length diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 798f3a5c94740..4f9eaacf67fe4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -55,7 +55,7 @@ class LogisticRegressionModel( this } - override def predictPoint(dataMatrix: Vector, weightMatrix: Vector, + override protected def predictPoint(dataMatrix: Vector, weightMatrix: Vector, intercept: Double) = { val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept val score = 1.0/ (1.0 + math.exp(-margin)) @@ -71,27 +71,27 @@ class LogisticRegressionModel( * NOTE: Labels used in Logistic Regression should be {0, 1} */ class LogisticRegressionWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var regParam: Double, - var miniBatchFraction: Double) + private var stepSize: Double, + private var numIterations: Int, + private var regParam: Double, + private var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable { - val gradient = new LogisticGradient() - val updater = new SimpleUpdater() + private val gradient = new LogisticGradient() + private val updater = new SimpleUpdater() override val optimizer = new GradientDescent(gradient, updater) .setStepSize(stepSize) .setNumIterations(numIterations) .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) - override val validators = List(DataValidators.classificationLabels) + override protected val validators = List(DataValidators.binaryLabelValidator) /** * Construct a LogisticRegression object with default parameters */ def this() = this(1.0, 100, 0.0, 1.0) - def createModel(weights: Vector, intercept: Double) = { + override protected def createModel(weights: Vector, intercept: Double) = { new LogisticRegressionModel(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index e956185319a69..5a45f12f1aa12 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.classification import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} +import org.apache.spark.annotation.Experimental import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vector @@ -27,11 +28,16 @@ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD /** + * :: Experimental :: + * * Model for Naive Bayes Classifiers. * - * @param pi Log of class priors, whose dimension is C. - * @param theta Log of class conditional probabilities, whose dimension is CxD. + * @param labels list of labels + * @param pi log of class priors, whose dimension is C, number of labels + * @param theta log of class conditional probabilities, whose dimension is C-by-D, + * where D is number of features */ +@Experimental class NaiveBayesModel( val labels: Array[Double], val pi: Array[Double], @@ -40,14 +46,17 @@ class NaiveBayesModel( private val brzPi = new BDV[Double](pi) private val brzTheta = new BDM[Double](theta.length, theta(0).length) - var i = 0 - while (i < theta.length) { - var j = 0 - while (j < theta(i).length) { - brzTheta(i, j) = theta(i)(j) - j += 1 + { + // Need to put an extra pair of braces to prevent Scala treating `i` as a member. + var i = 0 + while (i < theta.length) { + var j = 0 + while (j < theta(i).length) { + brzTheta(i, j) = theta(i)(j) + j += 1 + } + i += 1 } - i += 1 } override def predict(testData: RDD[Vector]): RDD[Double] = testData.map(predict) @@ -65,7 +74,7 @@ class NaiveBayesModel( * document classification. By making every vector a 0-1 vector, it can also be used as * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). */ -class NaiveBayes private (var lambda: Double) extends Serializable with Logging { +class NaiveBayes private (private var lambda: Double) extends Serializable with Logging { def this() = this(1.0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index e31a08899f8bc..956654b1fe90a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -55,7 +55,9 @@ class SVMModel( this } - override def predictPoint(dataMatrix: Vector, weightMatrix: Vector, + override protected def predictPoint( + dataMatrix: Vector, + weightMatrix: Vector, intercept: Double) = { val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept threshold match { @@ -70,28 +72,27 @@ class SVMModel( * NOTE: Labels used in SVM should be {0, 1}. */ class SVMWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var regParam: Double, - var miniBatchFraction: Double) + private var stepSize: Double, + private var numIterations: Int, + private var regParam: Double, + private var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[SVMModel] with Serializable { - val gradient = new HingeGradient() - val updater = new SquaredL2Updater() + private val gradient = new HingeGradient() + private val updater = new SquaredL2Updater() override val optimizer = new GradientDescent(gradient, updater) .setStepSize(stepSize) .setNumIterations(numIterations) .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) - - override val validators = List(DataValidators.classificationLabels) + override protected val validators = List(DataValidators.binaryLabelValidator) /** * Construct a SVM object with default parameters */ def this() = this(1.0, 100, 1.0, 1.0) - def createModel(weights: Vector, intercept: Double) = { + override protected def createModel(weights: Vector, intercept: Double) = { new SVMModel(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index a78503df3134d..8f565eb60a60f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{DenseVector => BDV, Vector => BV, norm => breezeNorm} +import org.apache.spark.annotation.Experimental import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.{Vector, Vectors} @@ -37,12 +38,17 @@ import org.apache.spark.util.random.XORShiftRandom * to it should be cached by the user. */ class KMeans private ( - var k: Int, - var maxIterations: Int, - var runs: Int, - var initializationMode: String, - var initializationSteps: Int, - var epsilon: Double) extends Serializable with Logging { + private var k: Int, + private var maxIterations: Int, + private var runs: Int, + private var initializationMode: String, + private var initializationSteps: Int, + private var epsilon: Double) extends Serializable with Logging { + + /** + * Constructs a KMeans instance with default parameters: {k: 2, maxIterations: 20, runs: 1, + * initializationMode: "k-means||", initializationSteps: 5, epsilon: 1e-4}. + */ def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4) /** Set the number of clusters to create (k). Default: 2. */ @@ -71,6 +77,8 @@ class KMeans private ( } /** + * :: Experimental :: + * * Set the number of runs of the algorithm to execute in parallel. We initialize the algorithm * this many times with random starting conditions (configured by the initialization mode), then * return the best clustering found over any run. Default: 1. @@ -316,8 +324,8 @@ object KMeans { data: RDD[Vector], k: Int, maxIterations: Int, - runs: Int = 1, - initializationMode: String = K_MEANS_PARALLEL): KMeansModel = { + runs: Int, + initializationMode: String): KMeansModel = { new KMeans().setK(k) .setMaxIterations(maxIterations) .setRuns(runs) @@ -325,6 +333,27 @@ object KMeans { .run(data) } + /** + * Trains a k-means model using specified parameters and the default values for unspecified. + */ + def train( + data: RDD[Vector], + k: Int, + maxIterations: Int): KMeansModel = { + train(data, k, maxIterations, 1, K_MEANS_PARALLEL) + } + + /** + * Trains a k-means model using specified parameters and the default values for unspecified. + */ + def train( + data: RDD[Vector], + k: Int, + maxIterations: Int, + runs: Int): KMeansModel = { + train(data, k, maxIterations, runs, K_MEANS_PARALLEL) + } + /** * Returns the index of the closest center to the given point, as well as the squared distance. */ @@ -369,6 +398,10 @@ object KMeans { MLUtils.fastSquaredDistance(v1.vector, v1.norm, v2.vector, v2.norm) } + /** + * :: Experimental :: + */ + @Experimental def main(args: Array[String]) { if (args.length < 4) { println("Usage: KMeans []") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 2cea58cd3fd22..99a849f1c66b1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -64,11 +64,13 @@ trait Vector extends Serializable { /** * Factory methods for [[org.apache.spark.mllib.linalg.Vector]]. + * We don't use the name `Vector` because Scala imports + * [[scala.collection.immutable.Vector]] by default. */ object Vectors { /** - * Creates a dense vector. + * Creates a dense vector from its values. */ @varargs def dense(firstValue: Double, otherValues: Double*): Vector = @@ -158,20 +160,21 @@ class DenseVector(val values: Array[Double]) extends Vector { /** * A sparse vector represented by an index array and an value array. * - * @param n size of the vector. + * @param size size of the vector. * @param indices index array, assume to be strictly increasing. * @param values value array, must have the same length as the index array. */ -class SparseVector(val n: Int, val indices: Array[Int], val values: Array[Double]) extends Vector { - - override def size: Int = n +class SparseVector( + override val size: Int, + val indices: Array[Int], + val values: Array[Double]) extends Vector { override def toString: String = { - "(" + n + "," + indices.zip(values).mkString("[", "," ,"]") + ")" + "(" + size + "," + indices.zip(values).mkString("[", "," ,"]") + ")" } override def toArray: Array[Double] = { - val data = new Array[Double](n) + val data = new Array[Double](size) var i = 0 val nnz = indices.length while (i < nnz) { @@ -181,5 +184,5 @@ class SparseVector(val n: Int, val indices: Array[Int], val values: Array[Double data } - private[mllib] override def toBreeze: BV[Double] = new BSV[Double](indices, values, n) + private[mllib] override def toBreeze: BV[Double] = new BSV[Double](indices, values, size) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala index 9194f657494b2..89d5c03d76c42 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} +import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vectors @@ -32,6 +33,8 @@ import org.apache.spark.mllib.linalg.Vectors case class MatrixEntry(i: Long, j: Long, value: Double) /** + * :: Experimental :: + * * Represents a matrix in coordinate format. * * @param entries matrix entries @@ -40,6 +43,7 @@ case class MatrixEntry(i: Long, j: Long, value: Double) * @param nCols number of columns. A non-positive value means unknown, and then the number of * columns will be determined by the max column index plus one. */ +@Experimental class CoordinateMatrix( val entries: RDD[MatrixEntry], private var nRows: Long, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala index 13f72a3c724ef..a0e26ce3bc465 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala @@ -19,8 +19,6 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} -import org.apache.spark.mllib.linalg.Matrix - /** * Represents a distributively stored matrix backed by one or more RDDs. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala index e110f070bd7c1..24c123ab7eb51 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala @@ -19,14 +19,22 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} +import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.linalg.SingularValueDecomposition -/** Represents a row of [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]]. */ +/** + * :: Experimental :: + * + * Represents a row of [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]]. + */ +@Experimental case class IndexedRow(index: Long, vector: Vector) /** + * :: Experimental :: + * * Represents a row-oriented [[org.apache.spark.mllib.linalg.distributed.DistributedMatrix]] with * indexed rows. * @@ -36,6 +44,7 @@ case class IndexedRow(index: Long, vector: Vector) * @param nCols number of columns. A non-positive value means unknown, and then the number of * columns will be determined by the size of the first row. */ +@Experimental class IndexedRowMatrix( val rows: RDD[IndexedRow], private var nRows: Long, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index f59811f18a68f..8d32c1a6dbba1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -23,11 +23,14 @@ import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, svd => brzSvd} import breeze.numerics.{sqrt => brzSqrt} import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg._ import org.apache.spark.rdd.RDD import org.apache.spark.Logging /** + * :: Experimental :: + * * Represents a row-oriented distributed Matrix with no meaningful row indices. * * @param rows rows stored as an RDD[Vector] @@ -36,6 +39,7 @@ import org.apache.spark.Logging * @param nCols number of columns. A non-positive value means unknown, and then the number of * columns will be determined by the size of the first row. */ +@Experimental class RowMatrix( val rows: RDD[Vector], private var nRows: Long, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 20654284965ed..1176dc9dbc08d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -19,11 +19,15 @@ package org.apache.spark.mllib.optimization import breeze.linalg.{axpy => brzAxpy} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.{Vectors, Vector} /** + * :: DeveloperApi :: + * * Class used to compute the gradient for a loss function, given a single data point. */ +@DeveloperApi abstract class Gradient extends Serializable { /** * Compute the gradient and loss given the features of a single data point. @@ -51,9 +55,12 @@ abstract class Gradient extends Serializable { } /** + * :: DeveloperApi :: + * * Compute gradient and loss for a logistic loss function, as used in binary classification. * See also the documentation for the precise formulation. */ +@DeveloperApi class LogisticGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val brzData = data.toBreeze @@ -92,11 +99,14 @@ class LogisticGradient extends Gradient { } /** + * :: DeveloperApi :: + * * Compute gradient and loss for a Least-squared loss function, as used in linear regression. * This is correct for the averaged least squares loss function (mean squared error) * L = 1/n ||A weights-y||^2 * See also the documentation for the precise formulation. */ +@DeveloperApi class LeastSquaresGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val brzData = data.toBreeze @@ -124,10 +134,13 @@ class LeastSquaresGradient extends Gradient { } /** + * :: DeveloperApi :: + * * Compute gradient and loss for a Hinge loss function, as used in SVM binary classification. * See also the documentation for the precise formulation. * NOTE: This assumes that the labels are {0,1} */ +@DeveloperApi class HingeGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val brzData = data.toBreeze diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index d0777ffd63ff8..04267d967dcad 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -19,18 +19,22 @@ package org.apache.spark.mllib.optimization import scala.collection.mutable.ArrayBuffer -import breeze.linalg.{Vector => BV, DenseVector => BDV} +import breeze.linalg.{DenseVector => BDV} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.{Vectors, Vector} /** + * :: DeveloperApi :: + * * Class used to solve an optimization problem using Gradient Descent. * @param gradient Gradient function to be used. * @param updater Updater to be used to update weights after every iteration. */ -class GradientDescent(var gradient: Gradient, var updater: Updater) +@DeveloperApi +class GradientDescent(private var gradient: Gradient, private var updater: Updater) extends Optimizer with Logging { private var stepSize: Double = 1.0 @@ -107,7 +111,12 @@ class GradientDescent(var gradient: Gradient, var updater: Updater) } -// Top-level method to run gradient descent. +/** + * :: DeveloperApi :: + * + * Top-level method to run gradient descent. + */ +@DeveloperApi object GradientDescent extends Logging { /** * Run stochastic gradient descent (SGD) in parallel using mini batches. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index f9ce908a5f3b0..0a313f3104b14 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -19,8 +19,15 @@ package org.apache.spark.mllib.optimization import org.apache.spark.rdd.RDD +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.Vector +/** + * :: DeveloperApi :: + * + * Trait for optimization problem solvers. + */ +@DeveloperApi trait Optimizer extends Serializable { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala index 3b7754cd7ac28..e67816796c6b1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -21,9 +21,12 @@ import scala.math._ import breeze.linalg.{norm => brzNorm, axpy => brzAxpy, Vector => BV} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.{Vectors, Vector} /** + * :: DeveloperApi :: + * * Class used to perform steps (weight update) using Gradient Descent methods. * * For general minimization problems, or for regularized problems of the form @@ -35,6 +38,7 @@ import org.apache.spark.mllib.linalg.{Vectors, Vector} * The updater is responsible to also perform the update coming from the * regularization term R(w) (if any regularization is used). */ +@DeveloperApi abstract class Updater extends Serializable { /** * Compute an updated value for weights given the gradient, stepSize, iteration number and @@ -59,9 +63,12 @@ abstract class Updater extends Serializable { } /** + * :: DeveloperApi :: + * * A simple updater for gradient descent *without* any regularization. * Uses a step-size decreasing with the square root of the number of iterations. */ +@DeveloperApi class SimpleUpdater extends Updater { override def compute( weightsOld: Vector, @@ -78,6 +85,8 @@ class SimpleUpdater extends Updater { } /** + * :: DeveloperApi :: + * * Updater for L1 regularized problems. * R(w) = ||w||_1 * Uses a step-size decreasing with the square root of the number of iterations. @@ -95,6 +104,7 @@ class SimpleUpdater extends Updater { * * Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal) */ +@DeveloperApi class L1Updater extends Updater { override def compute( weightsOld: Vector, @@ -120,10 +130,13 @@ class L1Updater extends Updater { } /** + * :: DeveloperApi :: + * * Updater for L2 regularized problems. * R(w) = 1/2 ||w||^2 * Uses a step-size decreasing with the square root of the number of iterations. */ +@DeveloperApi class SquaredL2Updater extends Updater { override def compute( weightsOld: Vector, 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 3124fac326d22..60cbb1c1e1d86 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 @@ -22,6 +22,10 @@ import scala.math.{abs, sqrt} import scala.util.Random import scala.util.Sorting +import com.esotericsoftware.kryo.Kryo +import org.jblas.{DoubleMatrix, SimpleBlas, Solve} + +import org.apache.spark.annotation.Experimental import org.apache.spark.broadcast.Broadcast import org.apache.spark.{Logging, HashPartitioner, Partitioner, SparkContext, SparkConf} import org.apache.spark.storage.StorageLevel @@ -29,10 +33,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.SparkContext._ -import com.esotericsoftware.kryo.Kryo -import org.jblas.{DoubleMatrix, SimpleBlas, Solve} - - /** * Out-link information for a user or product block. This includes the original user/product IDs * of the elements within this block, and the list of destination blocks that each user or @@ -90,14 +90,19 @@ case class Rating(val user: Int, val product: Int, val rating: Double) * 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, - var seed: Long = System.nanoTime() + private var numBlocks: Int, + private var rank: Int, + private var iterations: Int, + private var lambda: Double, + private var implicitPrefs: Boolean, + private var alpha: Double, + private var seed: Long = System.nanoTime() ) extends Serializable with Logging { + + /** + * Constructs an ALS instance with default parameters: {numBlocks: -1, rank: 10, iterations: 10, + * lambda: 0.01, implicitPrefs: false, alpha: 1.0}. + */ def this() = this(-1, 10, 10, 0.01, false, 1.0) /** @@ -127,11 +132,18 @@ class ALS private ( this } + /** Sets whether to use implicit preference. Default: false. */ def setImplicitPrefs(implicitPrefs: Boolean): ALS = { this.implicitPrefs = implicitPrefs this } + /** + * :: Experimental :: + * + * Sets the constant used in computing confidence in implicit ALS. Default: 1.0. + */ + @Experimental def setAlpha(alpha: Double): ALS = { this.alpha = alpha this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 443fc5de5bf04..e05224fc7caf2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -17,13 +17,14 @@ package org.apache.spark.mllib.recommendation +import org.jblas._ + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.mllib.api.python.PythonMLLibAPI -import org.jblas._ -import org.apache.spark.api.java.JavaRDD - /** * Model representing the result of matrix factorization. @@ -68,6 +69,8 @@ class MatrixFactorizationModel( } /** + * :: DeveloperApi :: + * * Predict the rating of many users for many products. * This is a Java stub for python predictAll() * diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 80dc0f12ff84f..c24f5afb99686 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.regression import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} +import org.apache.spark.annotation.Experimental import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ @@ -79,7 +80,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] protected val validators: Seq[RDD[LabeledPoint] => Boolean] = List() - val optimizer: Optimizer + /** The optimizer to solve the problem. */ + def optimizer: Optimizer /** Whether to add intercept (default: true). */ protected var addIntercept: Boolean = true @@ -100,8 +102,11 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] } /** + * :: Experimental :: + * * Set if the algorithm should validate data before training. Default true. */ + @Experimental def setValidateData(validateData: Boolean): this.type = { this.validateData = validateData this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index 25920d0dc976e..5f0812fd2e0eb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -52,15 +52,16 @@ class LassoModel( * See also the documentation for the precise formulation. */ class LassoWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var regParam: Double, - var miniBatchFraction: Double) + private var stepSize: Double, + private var numIterations: Int, + private var regParam: Double, + private var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[LassoModel] with Serializable { - val gradient = new LeastSquaresGradient() - val updater = new L1Updater() - @transient val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) + private val gradient = new LeastSquaresGradient() + private val updater = new L1Updater() + override val optimizer = new GradientDescent(gradient, updater) + .setStepSize(stepSize) .setNumIterations(numIterations) .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) @@ -69,7 +70,8 @@ class LassoWithSGD private ( super.setIntercept(false) /** - * Construct a Lasso object with default parameters + * Construct a Lasso object with default parameters: {stepSize: 1.0, numIterations: 100, + * regParam: 1.0, miniBatchFraction: 1.0}. */ def this() = this(1.0, 100, 1.0, 1.0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 9ed927994e795..228fa8db3e721 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -52,19 +52,21 @@ class LinearRegressionModel( * See also the documentation for the precise formulation. */ class LinearRegressionWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var miniBatchFraction: Double) + private var stepSize: Double, + private var numIterations: Int, + private var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[LinearRegressionModel] with Serializable { - val gradient = new LeastSquaresGradient() - val updater = new SimpleUpdater() - val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) + private val gradient = new LeastSquaresGradient() + private val updater = new SimpleUpdater() + override val optimizer = new GradientDescent(gradient, updater) + .setStepSize(stepSize) .setNumIterations(numIterations) .setMiniBatchFraction(miniBatchFraction) /** - * Construct a LinearRegression object with default parameters + * Construct a LinearRegression object with default parameters: {stepSize: 1.0, + * numIterations: 100, miniBatchFraction: 1.0}. */ def this() = this(1.0, 100, 1.0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index 1f17d2107f940..e702027c7c170 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -52,16 +52,17 @@ class RidgeRegressionModel( * See also the documentation for the precise formulation. */ class RidgeRegressionWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var regParam: Double, - var miniBatchFraction: Double) - extends GeneralizedLinearAlgorithm[RidgeRegressionModel] with Serializable { + private var stepSize: Double, + private var numIterations: Int, + private var regParam: Double, + private var miniBatchFraction: Double) + extends GeneralizedLinearAlgorithm[RidgeRegressionModel] with Serializable { - val gradient = new LeastSquaresGradient() - val updater = new SquaredL2Updater() + private val gradient = new LeastSquaresGradient() + private val updater = new SquaredL2Updater() - @transient val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) + override val optimizer = new GradientDescent(gradient, updater) + .setStepSize(stepSize) .setNumIterations(numIterations) .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) @@ -70,7 +71,8 @@ class RidgeRegressionWithSGD private ( super.setIntercept(false) /** - * Construct a RidgeRegression object with default parameters + * Construct a RidgeRegression object with default parameters: {stepSize: 1.0, numIterations: 100, + * regParam: 1.0, miniBatchFraction: 1.0}. */ def this() = this(1.0, 100, 1.0, 1.0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index dee9594a9dd79..c8a966cd5f5a8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.tree import scala.util.control.Breaks._ +import org.apache.spark.annotation.Experimental import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ import org.apache.spark.mllib.regression.LabeledPoint @@ -33,13 +34,16 @@ import org.apache.spark.util.random.XORShiftRandom import org.apache.spark.mllib.linalg.{Vector, Vectors} /** + * :: Experimental :: + * * A class that implements a decision tree algorithm for classification and regression. It * supports both continuous and categorical features. * @param strategy The configuration parameters for the tree algorithm which specify the type * of algorithm (classification, regression, etc.), feature type (continuous, * categorical), depth of the tree, quantile calculation strategy, etc. */ -class DecisionTree private(val strategy: Strategy) extends Serializable with Logging { +@Experimental +class DecisionTree (private val strategy: Strategy) extends Serializable with Logging { /** * Method to train a decision tree model over an RDD @@ -1024,7 +1028,7 @@ object DecisionTree extends Serializable with Logging { } } - val usage = """ + private val usage = """ Usage: DecisionTreeRunner [slices] --algo --trainDataDir path --testDataDir path --maxDepth num [--impurity ] [--maxBins num] @@ -1113,7 +1117,7 @@ object DecisionTree extends Serializable with Logging { * @return An RDD of LabeledPoint. Each labeled point has two elements: the first element is * the label, and the second element represents the feature values (an array of Double). */ - def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { + private def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { sc.textFile(dir).map { line => val parts = line.trim().split(",") val label = parts(0).toDouble diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala index 2dd1f0f27b8f5..017f84f3b9e8b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala @@ -17,9 +17,14 @@ package org.apache.spark.mllib.tree.configuration +import org.apache.spark.annotation.Experimental + /** + * :: Experimental :: + * * Enum to select the algorithm for the decision tree */ +@Experimental object Algo extends Enumeration { type Algo = Value val Classification, Regression = Value diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala index 09ee0586c58fa..c0254c32c2dce 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala @@ -17,9 +17,14 @@ package org.apache.spark.mllib.tree.configuration +import org.apache.spark.annotation.Experimental + /** + * :: Experimental :: + * * Enum to describe whether a feature is "continuous" or "categorical" */ +@Experimental object FeatureType extends Enumeration { type FeatureType = Value val Continuous, Categorical = Value diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala index 2457a480c2a14..b3e8b224beeaa 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala @@ -17,9 +17,14 @@ package org.apache.spark.mllib.tree.configuration +import org.apache.spark.annotation.Experimental + /** + * :: Experimental :: + * * Enum for selecting the quantile calculation strategy */ +@Experimental object QuantileStrategy extends Enumeration { type QuantileStrategy = Value val Sort, MinMax, ApproxHist = Value diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index df565f3eb8859..482faaa9e7256 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -17,11 +17,14 @@ package org.apache.spark.mllib.tree.configuration +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.tree.impurity.Impurity import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ /** + * :: Experimental :: + * * Stores all the configuration options for tree construction * @param algo classification or regression * @param impurity criterion used for information gain calculation @@ -34,10 +37,11 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * 1, 2, ... , k-1. It's important to note that features are * zero-indexed. */ +@Experimental class Strategy ( val algo: Algo, val impurity: Impurity, val maxDepth: Int, val maxBins: Int = 100, val quantileCalculationStrategy: QuantileStrategy = Sort, - val categoricalFeaturesInfo: Map[Int,Int] = Map[Int,Int]()) extends Serializable + val categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int]()) extends Serializable diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index b93995fcf9441..55c43f2fcf9c5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -17,31 +17,39 @@ package org.apache.spark.mllib.tree.impurity +import org.apache.spark.annotation.{DeveloperApi, Experimental} + /** + * :: Experimental :: + * * Class for calculating [[http://en.wikipedia.org/wiki/Binary_entropy_function entropy]] during * binary classification. */ +@Experimental object Entropy extends Impurity { - def log2(x: Double) = scala.math.log(x) / scala.math.log(2) + private[tree] def log2(x: Double) = scala.math.log(x) / scala.math.log(2) /** + * :: DeveloperApi :: + * * entropy calculation * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 * @return entropy value */ - def calculate(c0: Double, c1: Double): Double = { - if (c0 == 0 || c1 == 0) { - 0 - } else { - val total = c0 + c1 - val f0 = c0 / total - val f1 = c1 / total - -(f0 * log2(f0)) - (f1 * log2(f1)) - } - } + @DeveloperApi + override def calculate(c0: Double, c1: Double): Double = { + if (c0 == 0 || c1 == 0) { + 0 + } else { + val total = c0 + c1 + val f0 = c0 / total + val f1 = c1 / total + -(f0 * log2(f0)) - (f1 * log2(f1)) + } + } - def calculate(count: Double, sum: Double, sumSquares: Double): Double = + override def calculate(count: Double, sum: Double, sumSquares: Double): Double = throw new UnsupportedOperationException("Entropy.calculate") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index c0407554a91b3..c923b8e8f4cf1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -17,19 +17,27 @@ package org.apache.spark.mllib.tree.impurity +import org.apache.spark.annotation.{DeveloperApi, Experimental} + /** + * :: Experimental :: + * * Class for calculating the * [[http://en.wikipedia.org/wiki/Decision_tree_learning#Gini_impurity Gini impurity]] * during binary classification. */ +@Experimental object Gini extends Impurity { /** + * :: DeveloperApi :: + * * Gini coefficient calculation * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 * @return Gini coefficient value */ + @DeveloperApi override def calculate(c0: Double, c1: Double): Double = { if (c0 == 0 || c1 == 0) { 0 @@ -41,6 +49,6 @@ object Gini extends Impurity { } } - def calculate(count: Double, sum: Double, sumSquares: Double): Double = + override def calculate(count: Double, sum: Double, sumSquares: Double): Double = throw new UnsupportedOperationException("Gini.calculate") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index a4069063af2ad..f407796596c6c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -17,26 +17,36 @@ package org.apache.spark.mllib.tree.impurity +import org.apache.spark.annotation.{DeveloperApi, Experimental} + /** + * :: Experimental :: + * * Trait for calculating information gain. */ +@Experimental trait Impurity extends Serializable { /** + * :: DeveloperApi :: + * * information calculation for binary classification * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 * @return information value */ + @DeveloperApi def calculate(c0 : Double, c1 : Double): Double /** + * :: DeveloperApi :: + * * information calculation for regression * @param count number of instances * @param sum sum of labels * @param sumSquares summation of squares of the labels * @return information value */ + @DeveloperApi def calculate(count: Double, sum: Double, sumSquares: Double): Double - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index b74577dcec167..2c64644f4ed0f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -17,19 +17,27 @@ package org.apache.spark.mllib.tree.impurity +import org.apache.spark.annotation.{DeveloperApi, Experimental} + /** + * :: Experimental :: + * * Class for calculating variance during regression */ +@Experimental object Variance extends Impurity { override def calculate(c0: Double, c1: Double): Double = throw new UnsupportedOperationException("Variance.calculate") /** + * :: DeveloperApi :: + * * variance calculation * @param count number of instances * @param sum sum of labels * @param sumSquares summation of squares of the labels */ + @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = { val squaredLoss = sumSquares - (sum * sum) / count squaredLoss / count diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala index a57faa13745f7..2d71e1e366069 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala @@ -30,4 +30,5 @@ import org.apache.spark.mllib.tree.configuration.FeatureType._ * @param featureType type of feature -- categorical or continuous * @param category categorical label value accepted in the bin */ +private[tree] case class Bin(lowSplit: Split, highSplit: Split, featureType: FeatureType, category: Double) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index a6dca84a2ce09..0f76f4a049057 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -17,15 +17,19 @@ package org.apache.spark.mllib.tree.model +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vector /** + * :: Experimental :: + * * Model to store the decision tree parameters * @param topNode root node * @param algo algorithm type -- classification or regression */ +@Experimental class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala index ebc9595eafef3..2deaf4ae8dcab 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala @@ -22,7 +22,7 @@ package org.apache.spark.mllib.tree.model * @param split split specifying the feature index, type and threshold * @param comparison integer specifying <,=,> */ -case class Filter(split: Split, comparison: Int) { +private[tree] case class Filter(split: Split, comparison: Int) { // Comparison -1,0,1 signifies <.=,> override def toString = " split = " + split + "comparison = " + comparison } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index 99bf79cf12e45..d36b58e92ced6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -17,7 +17,11 @@ package org.apache.spark.mllib.tree.model +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: + * * Information gain statistics for each split * @param gain information gain value * @param impurity current node impurity @@ -25,6 +29,7 @@ package org.apache.spark.mllib.tree.model * @param rightImpurity right node impurity * @param predict predicted value */ +@DeveloperApi class InformationGainStats( val gain: Double, val impurity: Double, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index aac3f9ce308f7..339972141498c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -17,11 +17,14 @@ package org.apache.spark.mllib.tree.model +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.linalg.Vector /** + * :: DeveloperApi :: + * * Node in a decision tree * @param id integer node id * @param predict predicted value at the node @@ -31,6 +34,7 @@ import org.apache.spark.mllib.linalg.Vector * @param rightNode right child * @param stats information gain stats */ +@DeveloperApi class Node ( val id: Int, val predict: Double, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index 4e64a81dda74e..8bbb343079b49 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -17,20 +17,24 @@ package org.apache.spark.mllib.tree.model +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType /** + * :: DeveloperApi :: + * * Split applied to a feature * @param feature feature index * @param threshold threshold for continuous feature * @param featureType type of feature -- categorical or continuous * @param categories accepted values for categorical variables */ +@DeveloperApi case class Split( feature: Int, threshold: Double, featureType: FeatureType, - categories: List[Double]){ + categories: List[Double]) { override def toString = "Feature = " + feature + ", threshold = " + threshold + ", featureType = " + featureType + @@ -42,7 +46,7 @@ case class Split( * @param feature feature index * @param featureType type of feature -- categorical or continuous */ -class DummyLowSplit(feature: Int, featureType: FeatureType) +private[tree] class DummyLowSplit(feature: Int, featureType: FeatureType) extends Split(feature, Double.MinValue, featureType, List()) /** @@ -50,7 +54,7 @@ class DummyLowSplit(feature: Int, featureType: FeatureType) * @param feature feature index * @param featureType type of feature -- categorical or continuous */ -class DummyHighSplit(feature: Int, featureType: FeatureType) +private[tree] class DummyHighSplit(feature: Int, featureType: FeatureType) extends Split(feature, Double.MaxValue, featureType, List()) /** @@ -59,6 +63,6 @@ class DummyHighSplit(feature: Int, featureType: FeatureType) * @param feature feature index * @param featureType type of feature -- categorical or continuous */ -class DummyCategoricalSplit(feature: Int, featureType: FeatureType) +private[tree] class DummyCategoricalSplit(feature: Int, featureType: FeatureType) extends Split(feature, Double.MaxValue, featureType, List()) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala index 8b55bce7c4bec..230c409e1be33 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -17,23 +17,25 @@ package org.apache.spark.mllib.util +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint /** + * :: DeveloperApi :: + * * A collection of methods used to validate data before applying ML algorithms. */ +@DeveloperApi object DataValidators extends Logging { /** * Function to check if labels used for classification are either zero or one. * - * @param data - input data set that needs to be checked - * * @return True if labels are all zero or one, false otherwise. */ - val classificationLabels: RDD[LabeledPoint] => Boolean = { data => + val binaryLabelValidator: RDD[LabeledPoint] => Boolean = { data => val numInvalid = data.filter(x => x.label != 1.0 && x.label != 0.0).count() if (numInvalid != 0) { logError("Classification labels should be 0 or 1. Found " + numInvalid + " invalid labels") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala index 9109189dff52f..e693d13703987 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala @@ -19,15 +19,18 @@ package org.apache.spark.mllib.util import scala.util.Random +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD /** + * :: DeveloperApi :: + * * Generate test data for KMeans. This class first chooses k cluster centers * from a d-dimensional Gaussian distribution scaled by factor r and then creates a Gaussian * cluster with scale 1 around each center. */ - +@DeveloperApi object KMeansDataGenerator { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 81e4eda2a68c4..140ff92869176 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -22,16 +22,20 @@ import scala.util.Random import org.jblas.DoubleMatrix +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint /** + * :: DeveloperApi :: + * * Generate sample data used for Linear Data. This class generates * uniformly random values for every feature and adds Gaussian noise with mean `eps` to the * response variable `Y`. */ +@DeveloperApi object LinearDataGenerator { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala index 61498dcc2be00..ca06b9ad58538 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -19,16 +19,19 @@ package org.apache.spark.mllib.util import scala.util.Random +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.Vectors /** + * :: DeveloperApi :: + * * Generate test data for LogisticRegression. This class chooses positive labels * with probability `probOne` and scales features for positive examples by `eps`. */ - +@DeveloperApi object LogisticRegressionDataGenerator { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index 348aba1dea5b6..3bd86d6813375 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -21,10 +21,13 @@ import scala.util.Random import org.jblas.DoubleMatrix +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD /** + * :: DeveloperApi :: + * * Generate RDD(s) containing data for Matrix Factorization. * * This method samples training entries according to the oversampling factor @@ -47,9 +50,8 @@ import org.apache.spark.rdd.RDD * test (Boolean) Whether to create testing RDD. * testSampFact (Double) Percentage of training data to use as test data. */ - -object MFDataGenerator{ - +@DeveloperApi +object MFDataGenerator { def main(args: Array[String]) { if (args.length < 2) { println("Usage: MFDataGenerator " + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 83d1bd3fd57fe..7f9804deaf33f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.util import breeze.linalg.{Vector => BV, DenseVector => BDV, SparseVector => BSV, squaredDistance => breezeSquaredDistance} +import org.apache.spark.annotation.Experimental import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint @@ -122,6 +123,8 @@ object MLUtils { loadLibSVMData(sc, path, labelParser, numFeatures, sc.defaultMinSplits) /** + * :: Experimental :: + * * Load labeled data from a file. The data format used here is * , ... * where , are feature values in Double and is the corresponding label as Double. @@ -131,6 +134,7 @@ object MLUtils { * @return An RDD of LabeledPoint. Each labeled point has two elements: the first element is * the label, and the second element represents the feature values (an array of Double). */ + @Experimental def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { sc.textFile(dir).map { line => val parts = line.split(',') @@ -141,6 +145,8 @@ object MLUtils { } /** + * :: Experimental :: + * * Save labeled data to a file. The data format used here is * , ... * where , are feature values in Double and is the corresponding label as Double. @@ -148,6 +154,7 @@ object MLUtils { * @param data An RDD of LabeledPoints containing data to be saved. * @param dir Directory to save the data. */ + @Experimental def saveLabeledData(data: RDD[LabeledPoint], dir: String) { val dataStr = data.map(x => x.label + "," + x.features.toArray.mkString(" ")) dataStr.saveAsTextFile(dir) @@ -165,7 +172,7 @@ object MLUtils { * xColMean - Row vector with mean for every column (or feature) of the input data * xColSd - Row vector standard deviation for every column (or feature) of the input data. */ - def computeStats( + private[mllib] def computeStats( data: RDD[LabeledPoint], numFeatures: Int, numExamples: Long): (Double, Vector, Vector) = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index e300c3dbe1fe0..87a6f2a0c3976 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -21,15 +21,19 @@ import scala.util.Random import org.jblas.DoubleMatrix +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint /** + * :: DeveloperApi :: + * * Generate sample data used for SVM. This class generates uniform random values * for the features and adds Gaussian noise with weight 0.1 to generate labels. */ +@DeveloperApi object SVMDataGenerator { def main(args: Array[String]) { From eb5f2b64230faa69a53815cb61bcc87aeb233d20 Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Wed, 9 Apr 2014 15:24:33 -0700 Subject: [PATCH 038/641] SPARK-1407 drain event queue before stopping event logger Author: Kan Zhang Closes #366 from kanzhang/SPARK-1407 and squashes the following commits: cd0629f [Kan Zhang] code refactoring and adding test b073ee6 [Kan Zhang] SPARK-1407 drain event queue before stopping event logger --- .../scala/org/apache/spark/SparkContext.scala | 4 +- .../spark/scheduler/LiveListenerBus.scala | 32 +++++++------ .../spark/scheduler/SparkListenerSuite.scala | 45 +++++++++++++++++++ .../apache/spark/examples/SparkHdfsLR.scala | 2 +- 4 files changed, 67 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index f7750514ae13d..76305237b03d5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -931,7 +931,6 @@ class SparkContext(config: SparkConf) extends Logging { /** Shut down the SparkContext. */ def stop() { ui.stop() - eventLogger.foreach(_.stop()) // Do this only if not stopped already - best case effort. // prevent NPE if stopped more than once. val dagSchedulerCopy = dagScheduler @@ -940,13 +939,14 @@ class SparkContext(config: SparkConf) extends Logging { metadataCleaner.cancel() cleaner.foreach(_.stop()) dagSchedulerCopy.stop() - listenerBus.stop() taskScheduler = null // TODO: Cache.stop()? env.stop() SparkEnv.set(null) ShuffleMapTask.clearCache() ResultTask.clearCache() + listenerBus.stop() + eventLogger.foreach(_.stop()) logInfo("Successfully stopped SparkContext") } else { logInfo("SparkContext already stopped") diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index 353a48661b0f7..76f3e327d60b8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -36,6 +36,22 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) private var queueFullErrorMessageLogged = false private var started = false + private val listenerThread = new Thread("SparkListenerBus") { + setDaemon(true) + override def run() { + while (true) { + val event = eventQueue.take + if (event == SparkListenerShutdown) { + // Get out of the while loop and shutdown the daemon thread + return + } + postToAll(event) + } + } + } + + // Exposed for testing + @volatile private[spark] var stopCalled = false /** * Start sending events to attached listeners. @@ -48,20 +64,8 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { if (started) { throw new IllegalStateException("Listener bus already started!") } + listenerThread.start() started = true - new Thread("SparkListenerBus") { - setDaemon(true) - override def run() { - while (true) { - val event = eventQueue.take - if (event == SparkListenerShutdown) { - // Get out of the while loop and shutdown the daemon thread - return - } - postToAll(event) - } - } - }.start() } def post(event: SparkListenerEvent) { @@ -93,9 +97,11 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { } def stop() { + stopCalled = true if (!started) { throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") } post(SparkListenerShutdown) + listenerThread.join() } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 7c843772bc2e0..dc704e07a81de 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import java.util.concurrent.Semaphore + import scala.collection.mutable import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} @@ -72,6 +74,49 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } } + test("bus.stop() waits for the event queue to completely drain") { + @volatile var drained = false + + // Tells the listener to stop blocking + val listenerWait = new Semaphore(1) + + // When stop has returned + val stopReturned = new Semaphore(1) + + class BlockingListener extends SparkListener { + override def onJobEnd(jobEnd: SparkListenerJobEnd) = { + listenerWait.acquire() + drained = true + } + } + + val bus = new LiveListenerBus + val blockingListener = new BlockingListener + + bus.addListener(blockingListener) + bus.start() + bus.post(SparkListenerJobEnd(0, JobSucceeded)) + + // the queue should not drain immediately + assert(!drained) + + new Thread("ListenerBusStopper") { + override def run() { + // stop() will block until notify() is called below + bus.stop() + stopReturned.release(1) + } + }.start() + + while (!bus.stopCalled) { + Thread.sleep(10) + } + + listenerWait.release() + stopReturned.acquire() + assert(drained) + } + test("basic creation of StageInfo") { val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index e698b9bf376e1..038afbcba80a3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -73,6 +73,6 @@ object SparkHdfsLR { } println("Final w: " + w) - System.exit(0) + sc.stop() } } From 0adc932add413a1754107b21d5ecfb38c0c3a4eb Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 9 Apr 2014 17:08:17 -0700 Subject: [PATCH 039/641] [SPARK-1357 (fix)] remove empty line after :: DeveloperApi/Experimental :: Remove empty line after :: DeveloperApi/Experimental :: in comments to make the original doc show up in the preview of the generated html docs. Thanks @andrewor14 ! Author: Xiangrui Meng Closes #373 from mengxr/api and squashes the following commits: 9c35bdc [Xiangrui Meng] remove the empty line after :: DeveloperApi/Experimental :: --- .../mllib/api/python/PythonMLLibAPI.scala | 1 - .../mllib/classification/NaiveBayes.scala | 1 - .../spark/mllib/clustering/KMeans.scala | 4 -- .../linalg/distributed/CoordinateMatrix.scala | 1 - .../linalg/distributed/IndexedRowMatrix.scala | 2 - .../mllib/linalg/distributed/RowMatrix.scala | 1 - .../spark/mllib/optimization/Gradient.scala | 4 -- .../mllib/optimization/GradientDescent.scala | 2 - .../spark/mllib/optimization/Optimizer.scala | 1 - .../spark/mllib/optimization/Updater.scala | 4 -- .../spark/mllib/recommendation/ALS.scala | 1 - .../MatrixFactorizationModel.scala | 1 - .../GeneralizedLinearAlgorithm.scala | 1 - .../spark/mllib/tree/DecisionTree.scala | 1 - .../spark/mllib/tree/configuration/Algo.scala | 1 - .../tree/configuration/FeatureType.scala | 1 - .../tree/configuration/QuantileStrategy.scala | 1 - .../mllib/tree/configuration/Strategy.scala | 1 - .../spark/mllib/tree/impurity/Entropy.scala | 2 - .../spark/mllib/tree/impurity/Gini.scala | 2 - .../spark/mllib/tree/impurity/Impurity.scala | 3 -- .../spark/mllib/tree/impurity/Variance.scala | 2 - .../mllib/tree/model/DecisionTreeModel.scala | 1 - .../tree/model/InformationGainStats.scala | 1 - .../apache/spark/mllib/tree/model/Node.scala | 1 - .../apache/spark/mllib/tree/model/Split.scala | 1 - .../spark/mllib/util/DataValidators.scala | 1 - .../mllib/util/KMeansDataGenerator.scala | 1 - .../mllib/util/LinearDataGenerator.scala | 1 - .../LogisticRegressionDataGenerator.scala | 1 - .../spark/mllib/util/MFDataGenerator.scala | 43 +++++++++---------- .../org/apache/spark/mllib/util/MLUtils.scala | 2 - .../spark/mllib/util/SVMDataGenerator.scala | 1 - 33 files changed, 21 insertions(+), 71 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index ae27c57799873..a6c049e517ee0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -30,7 +30,6 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: - * * The Java stubs necessary for the Python mllib bindings. */ @DeveloperApi diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index 5a45f12f1aa12..18658850a2f64 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -29,7 +29,6 @@ import org.apache.spark.rdd.RDD /** * :: Experimental :: - * * Model for Naive Bayes Classifiers. * * @param labels list of labels diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 8f565eb60a60f..90cf8525df523 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -78,7 +78,6 @@ class KMeans private ( /** * :: Experimental :: - * * Set the number of runs of the algorithm to execute in parallel. We initialize the algorithm * this many times with random starting conditions (configured by the initialization mode), then * return the best clustering found over any run. Default: 1. @@ -398,9 +397,6 @@ object KMeans { MLUtils.fastSquaredDistance(v1.vector, v1.norm, v2.vector, v2.norm) } - /** - * :: Experimental :: - */ @Experimental def main(args: Array[String]) { if (args.length < 4) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala index 89d5c03d76c42..56b8fdcda66eb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -34,7 +34,6 @@ case class MatrixEntry(i: Long, j: Long, value: Double) /** * :: Experimental :: - * * Represents a matrix in coordinate format. * * @param entries matrix entries diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala index 24c123ab7eb51..132b3af72d9ce 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala @@ -26,7 +26,6 @@ import org.apache.spark.mllib.linalg.SingularValueDecomposition /** * :: Experimental :: - * * Represents a row of [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]]. */ @Experimental @@ -34,7 +33,6 @@ case class IndexedRow(index: Long, vector: Vector) /** * :: Experimental :: - * * Represents a row-oriented [[org.apache.spark.mllib.linalg.distributed.DistributedMatrix]] with * indexed rows. * diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 8d32c1a6dbba1..f65f43dd3007b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -30,7 +30,6 @@ import org.apache.spark.Logging /** * :: Experimental :: - * * Represents a row-oriented distributed Matrix with no meaningful row indices. * * @param rows rows stored as an RDD[Vector] diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 1176dc9dbc08d..679842f831c2a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * :: DeveloperApi :: - * * Class used to compute the gradient for a loss function, given a single data point. */ @DeveloperApi @@ -56,7 +55,6 @@ abstract class Gradient extends Serializable { /** * :: DeveloperApi :: - * * Compute gradient and loss for a logistic loss function, as used in binary classification. * See also the documentation for the precise formulation. */ @@ -100,7 +98,6 @@ class LogisticGradient extends Gradient { /** * :: DeveloperApi :: - * * Compute gradient and loss for a Least-squared loss function, as used in linear regression. * This is correct for the averaged least squares loss function (mean squared error) * L = 1/n ||A weights-y||^2 @@ -135,7 +132,6 @@ class LeastSquaresGradient extends Gradient { /** * :: DeveloperApi :: - * * Compute gradient and loss for a Hinge loss function, as used in SVM binary classification. * See also the documentation for the precise formulation. * NOTE: This assumes that the labels are {0,1} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 04267d967dcad..f60417f21d4b9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -28,7 +28,6 @@ import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * :: DeveloperApi :: - * * Class used to solve an optimization problem using Gradient Descent. * @param gradient Gradient function to be used. * @param updater Updater to be used to update weights after every iteration. @@ -113,7 +112,6 @@ class GradientDescent(private var gradient: Gradient, private var updater: Updat /** * :: DeveloperApi :: - * * Top-level method to run gradient descent. */ @DeveloperApi diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index 0a313f3104b14..e41d9bbe18c37 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.linalg.Vector /** * :: DeveloperApi :: - * * Trait for optimization problem solvers. */ @DeveloperApi diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala index e67816796c6b1..3ed3a5b9b3843 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -26,7 +26,6 @@ import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * :: DeveloperApi :: - * * Class used to perform steps (weight update) using Gradient Descent methods. * * For general minimization problems, or for regularized problems of the form @@ -64,7 +63,6 @@ abstract class Updater extends Serializable { /** * :: DeveloperApi :: - * * A simple updater for gradient descent *without* any regularization. * Uses a step-size decreasing with the square root of the number of iterations. */ @@ -86,7 +84,6 @@ class SimpleUpdater extends Updater { /** * :: DeveloperApi :: - * * Updater for L1 regularized problems. * R(w) = ||w||_1 * Uses a step-size decreasing with the square root of the number of iterations. @@ -131,7 +128,6 @@ class L1Updater extends Updater { /** * :: DeveloperApi :: - * * Updater for L2 regularized problems. * R(w) = 1/2 ||w||^2 * Uses a step-size decreasing with the square root of the number of iterations. 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 60cbb1c1e1d86..5cc47de8ffdfc 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 @@ -140,7 +140,6 @@ class ALS private ( /** * :: Experimental :: - * * Sets the constant used in computing confidence in implicit ALS. Default: 1.0. */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index e05224fc7caf2..471546cd82c7d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -70,7 +70,6 @@ class MatrixFactorizationModel( /** * :: DeveloperApi :: - * * Predict the rating of many users for many products. * This is a Java stub for python predictAll() * diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index c24f5afb99686..3bd0017aa196a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -103,7 +103,6 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] /** * :: Experimental :: - * * Set if the algorithm should validate data before training. Default true. */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index c8a966cd5f5a8..3019447ce4cd9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -35,7 +35,6 @@ import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * :: Experimental :: - * * A class that implements a decision tree algorithm for classification and regression. It * supports both continuous and categorical features. * @param strategy The configuration parameters for the tree algorithm which specify the type diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala index 017f84f3b9e8b..79a01f58319e8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.Experimental /** * :: Experimental :: - * * Enum to select the algorithm for the decision tree */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala index c0254c32c2dce..f4c877232750f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.Experimental /** * :: Experimental :: - * * Enum to describe whether a feature is "continuous" or "categorical" */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala index b3e8b224beeaa..7da976e55a722 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.Experimental /** * :: Experimental :: - * * Enum for selecting the quantile calculation strategy */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 482faaa9e7256..8767aca47cd5a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ /** * :: Experimental :: - * * Stores all the configuration options for tree construction * @param algo classification or regression * @param impurity criterion used for information gain calculation diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index 55c43f2fcf9c5..60f43e9278d2a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} /** * :: Experimental :: - * * Class for calculating [[http://en.wikipedia.org/wiki/Binary_entropy_function entropy]] during * binary classification. */ @@ -32,7 +31,6 @@ object Entropy extends Impurity { /** * :: DeveloperApi :: - * * entropy calculation * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index c923b8e8f4cf1..c51d76d9b4c5b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} /** * :: Experimental :: - * * Class for calculating the * [[http://en.wikipedia.org/wiki/Decision_tree_learning#Gini_impurity Gini impurity]] * during binary classification. @@ -31,7 +30,6 @@ object Gini extends Impurity { /** * :: DeveloperApi :: - * * Gini coefficient calculation * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index f407796596c6c..8eab247cf0932 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} /** * :: Experimental :: - * * Trait for calculating information gain. */ @Experimental @@ -29,7 +28,6 @@ trait Impurity extends Serializable { /** * :: DeveloperApi :: - * * information calculation for binary classification * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 @@ -40,7 +38,6 @@ trait Impurity extends Serializable { /** * :: DeveloperApi :: - * * information calculation for regression * @param count number of instances * @param sum sum of labels diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index 2c64644f4ed0f..47d07122af30f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} /** * :: Experimental :: - * * Class for calculating variance during regression */ @Experimental @@ -31,7 +30,6 @@ object Variance extends Impurity { /** * :: DeveloperApi :: - * * variance calculation * @param count number of instances * @param sum sum of labels diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index 0f76f4a049057..bf692ca8c4bd7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.linalg.Vector /** * :: Experimental :: - * * Model to store the decision tree parameters * @param topNode root node * @param algo algorithm type -- classification or regression diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index d36b58e92ced6..cc8a24cce9614 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: - * * Information gain statistics for each split * @param gain information gain value * @param impurity current node impurity diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 339972141498c..682f213f411a7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.linalg.Vector /** * :: DeveloperApi :: - * * Node in a decision tree * @param id integer node id * @param predict predicted value at the node diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index 8bbb343079b49..d7ffd386c05ee 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -22,7 +22,6 @@ import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType /** * :: DeveloperApi :: - * * Split applied to a feature * @param feature feature index * @param threshold threshold for continuous feature diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala index 230c409e1be33..45f95482a1def 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.regression.LabeledPoint /** * :: DeveloperApi :: - * * A collection of methods used to validate data before applying ML algorithms. */ @DeveloperApi diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala index e693d13703987..6eaebaf7dba9f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala @@ -25,7 +25,6 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: - * * Generate test data for KMeans. This class first chooses k cluster centers * from a d-dimensional Gaussian distribution scaled by factor r and then creates a Gaussian * cluster with scale 1 around each center. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 140ff92869176..c8e160d00c2d6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -30,7 +30,6 @@ import org.apache.spark.mllib.regression.LabeledPoint /** * :: DeveloperApi :: - * * Generate sample data used for Linear Data. This class generates * uniformly random values for every feature and adds Gaussian noise with mean `eps` to the * response variable `Y`. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala index ca06b9ad58538..c82cd8fd4641c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -27,7 +27,6 @@ import org.apache.spark.mllib.linalg.Vectors /** * :: DeveloperApi :: - * * Generate test data for LogisticRegression. This class chooses positive labels * with probability `probOne` and scales features for positive examples by `eps`. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index 3bd86d6813375..3f413faca6bb4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -27,29 +27,28 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: + * Generate RDD(s) containing data for Matrix Factorization. * -* Generate RDD(s) containing data for Matrix Factorization. -* -* This method samples training entries according to the oversampling factor -* 'trainSampFact', which is a multiplicative factor of the number of -* degrees of freedom of the matrix: rank*(m+n-rank). -* -* It optionally samples entries for a testing matrix using -* 'testSampFact', the percentage of the number of training entries -* to use for testing. -* -* This method takes the following inputs: -* sparkMaster (String) The master URL. -* outputPath (String) Directory to save output. -* m (Int) Number of rows in data matrix. -* n (Int) Number of columns in data matrix. -* rank (Int) Underlying rank of data matrix. -* trainSampFact (Double) Oversampling factor. -* noise (Boolean) Whether to add gaussian noise to training data. -* sigma (Double) Standard deviation of added gaussian noise. -* test (Boolean) Whether to create testing RDD. -* testSampFact (Double) Percentage of training data to use as test data. -*/ + * This method samples training entries according to the oversampling factor + * 'trainSampFact', which is a multiplicative factor of the number of + * degrees of freedom of the matrix: rank*(m+n-rank). + * + * It optionally samples entries for a testing matrix using + * 'testSampFact', the percentage of the number of training entries + * to use for testing. + * + * This method takes the following inputs: + * sparkMaster (String) The master URL. + * outputPath (String) Directory to save output. + * m (Int) Number of rows in data matrix. + * n (Int) Number of columns in data matrix. + * rank (Int) Underlying rank of data matrix. + * trainSampFact (Double) Oversampling factor. + * noise (Boolean) Whether to add gaussian noise to training data. + * sigma (Double) Standard deviation of added gaussian noise. + * test (Boolean) Whether to create testing RDD. + * testSampFact (Double) Percentage of training data to use as test data. + */ @DeveloperApi object MFDataGenerator { def main(args: Array[String]) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 7f9804deaf33f..ac2360c429e2b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -124,7 +124,6 @@ object MLUtils { /** * :: Experimental :: - * * Load labeled data from a file. The data format used here is * , ... * where , are feature values in Double and is the corresponding label as Double. @@ -146,7 +145,6 @@ object MLUtils { /** * :: Experimental :: - * * Save labeled data to a file. The data format used here is * , ... * where , are feature values in Double and is the corresponding label as Double. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index 87a6f2a0c3976..ba8190b0e07e8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -29,7 +29,6 @@ import org.apache.spark.mllib.regression.LabeledPoint /** * :: DeveloperApi :: - * * Generate sample data used for SVM. This class generates uniform random values * for the features and adds Gaussian noise with weight 0.1 to generate labels. */ From 8ca3b2bc90a63b23a03f339e390174cd7a672b40 Mon Sep 17 00:00:00 2001 From: William Benton Date: Wed, 9 Apr 2014 18:56:27 -0700 Subject: [PATCH 040/641] SPARK-729: Closures not always serialized at capture time [SPARK-729](https://spark-project.atlassian.net/browse/SPARK-729) concerns when free variables in closure arguments to transformations are captured. Currently, it is possible for closures to get the environment in which they are serialized (not the environment in which they are created). There are a few possible approaches to solving this problem and this PR will discuss some of them. The approach I took has the advantage of being simple, obviously correct, and minimally-invasive, but it preserves something that has been bothering me about Spark's closure handling, so I'd like to discuss an alternative and get some feedback on whether or not it is worth pursuing. ## What I did The basic approach I took depends on the work I did for #143, and so this PR is based atop that. Specifically: #143 modifies `ClosureCleaner.clean` to preemptively determine whether or not closures are serializable immediately upon closure cleaning (rather than waiting for an job involving that closure to be scheduled). Thus non-serializable closure exceptions will be triggered by the line defining the closure rather than triggered where the closure is used. Since the easiest way to determine whether or not a closure is serializable is to attempt to serialize it, the code in #143 is creating a serialized closure as part of `ClosureCleaner.clean`. `clean` currently modifies its argument, but the method in `SparkContext` that wraps it to return a value (a reference to the modified-in-place argument). This branch modifies `ClosureCleaner.clean` so that it returns a value: if it is cleaning a serializable closure, it returns the result of deserializing its serialized argument; therefore it is returning a closure with an environment captured at cleaning time. `SparkContext.clean` then returns the result of `ClosureCleaner.clean`, rather than a reference to its modified-in-place argument. I've added tests for this behavior (777a1bc). The pull request as it stands, given the changes in #143, is nearly trivial. There is some overhead from deserializing the closure, but it is minimal and the benefit of obvious operational correctness (vs. a more sophisticated but harder-to-validate transformation in `ClosureCleaner`) seems pretty important. I think this is a fine way to solve this problem, but it's not perfect. ## What we might want to do The thing that has been bothering me about Spark's handling of closures is that it seems like we should be able to statically ensure that cleaning and serialization happen exactly once for a given closure. If we serialize a closure in order to determine whether or not it is serializable, we should be able to hang on to the generated byte buffer and use it instead of re-serializing the closure later. By replacing closures with instances of a sum type that encodes whether or not a closure has been cleaned or serialized, we could handle clean, to-be-cleaned, and serialized closures separately with case matches. Here's a somewhat-concrete sketch (taken from my git stash) of what this might look like: ```scala package org.apache.spark.util import java.nio.ByteBuffer import scala.reflect.ClassManifest sealed abstract class ClosureBox[T] { def func: T } final case class RawClosure[T](func: T) extends ClosureBox[T] {} final case class CleanedClosure[T](func: T) extends ClosureBox[T] {} final case class SerializedClosure[T](func: T, bytebuf: ByteBuffer) extends ClosureBox[T] {} object ClosureBoxImplicits { implicit def closureBoxFromFunc[T <: AnyRef](fun: T) = new RawClosure[T](fun) } ``` With these types declared, we'd be able to change `ClosureCleaner.clean` to take a `ClosureBox[T=>U]` (possibly generated by implicit conversion) and return a `ClosureBox[T=>U]` (either a `CleanedClosure[T=>U]` or a `SerializedClosure[T=>U]`, depending on whether or not serializability-checking was enabled) instead of a `T=>U`. A case match could thus short-circuit cleaning or serializing closures that had already been cleaned or serialized (both in `ClosureCleaner` and in the closure serializer). Cleaned-and-serialized closures would be represented by a boxed tuple of the original closure and a serialized copy (complete with an environment quiesced at transformation time). Additional implicit conversions could convert from `ClosureBox` instances to the underlying function type where appropriate. Tracking this sort of state in the type system seems like the right thing to do to me. ### Why we might not want to do that _It's pretty invasive._ Every function type used by every `RDD` subclass would have to change to reflect that they expected a `ClosureBox[T=>U]` instead of a `T=>U`. This obscures what's going on and is not a little ugly. Although I really like the idea of using the type system to enforce the clean-or-serialize once discipline, it might not be worth adding another layer of types (even if we could hide some of the extra boilerplate with judicious application of implicit conversions). _It statically guarantees a property whose absence is unlikely to cause any serious problems as it stands._ It appears that all closures are currently dynamically cleaned once and it's not obvious that repeated closure-cleaning is likely to be a problem in the future. Furthermore, serializing closures is relatively cheap, so doing it once to check for serialization and once again to actually ship them across the wire doesn't seem like a big deal. Taken together, these seem like a high price to pay for statically guaranteeing that closures are operated upon only once. ## Other possibilities I felt like the serialize-and-deserialize approach was best due to its obvious simplicity. But it would be possible to do a more sophisticated transformation within `ClosureCleaner.clean`. It might also be possible for `clean` to modify its argument in a way so that whether or not a given closure had been cleaned would be apparent upon inspection; this would buy us some of the operational benefits of the `ClosureBox` approach but not the static cleanliness. I'm interested in any feedback or discussion on whether or not the problems with the type-based approach indeed outweigh the advantage, as well as of approaches to this issue and to closure handling in general. Author: William Benton Closes #189 from willb/spark-729 and squashes the following commits: f4cafa0 [William Benton] Stylistic changes and cleanups b3d9c86 [William Benton] Fixed style issues in tests 9b56ce0 [William Benton] Added array-element capture test 97e9d91 [William Benton] Split closure-serializability failure tests 12ef6e3 [William Benton] Skip proactive closure capture for runJob 8ee3ee7 [William Benton] Predictable closure environment capture 12c63a7 [William Benton] Added tests for variable capture in closures d6e8dd6 [William Benton] Don't check serializability of DStream transforms. 4ecf841 [William Benton] Make proactive serializability checking optional. d8df3db [William Benton] Adds proactive closure-serializablilty checking 21b4b06 [William Benton] Test cases for SPARK-897. d5947b3 [William Benton] Ensure assertions in Graph.apply are asserted. --- .../scala/org/apache/spark/SparkContext.scala | 16 +++- .../main/scala/org/apache/spark/rdd/RDD.scala | 6 +- .../apache/spark/util/ClosureCleaner.scala | 21 ++++- .../scala/org/apache/spark/FailureSuite.scala | 17 +++- .../ProactiveClosureSerializationSuite.scala | 94 +++++++++++++++++++ .../spark/util/ClosureCleanerSuite.scala | 68 ++++++++++++++ .../org/apache/spark/graphx/GraphSuite.scala | 2 +- .../spark/streaming/dstream/DStream.scala | 8 +- 8 files changed, 218 insertions(+), 14 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 76305237b03d5..545807ffbce55 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1002,7 +1002,9 @@ class SparkContext(config: SparkConf) extends Logging { require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p") } val callSite = getCallSite - val cleanedFunc = clean(func) + // There's no need to check this function for serializability, + // since it will be run right away. + val cleanedFunc = clean(func, false) logInfo("Starting job: " + callSite) val start = System.nanoTime dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal, @@ -1135,14 +1137,18 @@ class SparkContext(config: SparkConf) extends Logging { def cancelAllJobs() { dagScheduler.cancelAllJobs() } - + /** * Clean a closure to make it ready to serialized and send to tasks * (removes unreferenced variables in $outer's, updates REPL variables) + * + * @param f closure to be cleaned and optionally serialized + * @param captureNow whether or not to serialize this closure and capture any free + * variables immediately; defaults to true. If this is set and f is not serializable, + * it will raise an exception. */ - private[spark] def clean[F <: AnyRef](f: F): F = { - ClosureCleaner.clean(f) - f + private[spark] def clean[F <: AnyRef : ClassTag](f: F, captureNow: Boolean = true): F = { + ClosureCleaner.clean(f, captureNow) } /** 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 3437b2cac19c2..e363ea777d8eb 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -660,14 +660,16 @@ abstract class RDD[T: ClassTag]( * Applies a function f to all elements of this RDD. */ def foreach(f: T => Unit) { - sc.runJob(this, (iter: Iterator[T]) => iter.foreach(f)) + val cleanF = sc.clean(f) + sc.runJob(this, (iter: Iterator[T]) => iter.foreach(cleanF)) } /** * Applies a function f to each partition of this RDD. */ def foreachPartition(f: Iterator[T] => Unit) { - sc.runJob(this, (iter: Iterator[T]) => f(iter)) + val cleanF = sc.clean(f) + sc.runJob(this, (iter: Iterator[T]) => cleanF(iter)) } /** diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index cdbbc65292188..e474b1a850d65 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -22,10 +22,14 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.collection.mutable.Map import scala.collection.mutable.Set +import scala.reflect.ClassTag + import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ import org.apache.spark.Logging +import org.apache.spark.SparkEnv +import org.apache.spark.SparkException private[spark] object ClosureCleaner extends Logging { // Get an ASM class reader for a given class from the JAR that loaded it @@ -101,7 +105,7 @@ private[spark] object ClosureCleaner extends Logging { } } - def clean(func: AnyRef) { + def clean[F <: AnyRef : ClassTag](func: F, captureNow: Boolean = true): F = { // TODO: cache outerClasses / innerClasses / accessedFields val outerClasses = getOuterClasses(func) val innerClasses = getInnerClasses(func) @@ -150,6 +154,21 @@ private[spark] object ClosureCleaner extends Logging { field.setAccessible(true) field.set(func, outer) } + + if (captureNow) { + cloneViaSerializing(func) + } else { + func + } + } + + private def cloneViaSerializing[T: ClassTag](func: T): T = { + try { + val serializer = SparkEnv.get.closureSerializer.newInstance() + serializer.deserialize[T](serializer.serialize[T](func)) + } catch { + case ex: Exception => throw new SparkException("Task not serializable: " + ex.toString) + } } private def instantiateClass(cls: Class[_], outer: AnyRef, inInterpreter: Boolean): AnyRef = { diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index 12dbebcb28644..4f9300419e6f8 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -107,7 +107,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { FailureSuiteState.clear() } - test("failure because task closure is not serializable") { + test("failure because closure in final-stage task is not serializable") { sc = new SparkContext("local[1,1]", "test") val a = new NonSerializable @@ -118,6 +118,13 @@ class FailureSuite extends FunSuite with LocalSparkContext { assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) + FailureSuiteState.clear() + } + + test("failure because closure in early-stage task is not serializable") { + sc = new SparkContext("local[1,1]", "test") + val a = new NonSerializable + // Non-serializable closure in an earlier stage val thrown1 = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() @@ -125,6 +132,13 @@ class FailureSuite extends FunSuite with LocalSparkContext { assert(thrown1.getClass === classOf[SparkException]) assert(thrown1.getMessage.contains("NotSerializableException")) + FailureSuiteState.clear() + } + + test("failure because closure in foreach task is not serializable") { + sc = new SparkContext("local[1,1]", "test") + val a = new NonSerializable + // Non-serializable closure in foreach function val thrown2 = intercept[SparkException] { sc.parallelize(1 to 10, 2).foreach(x => println(a)) @@ -135,5 +149,6 @@ class FailureSuite extends FunSuite with LocalSparkContext { FailureSuiteState.clear() } + // TODO: Need to add tests with shuffle fetch failures. } diff --git a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala new file mode 100644 index 0000000000000..76662264e7e94 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala @@ -0,0 +1,94 @@ +/* + * 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.io.NotSerializableException + +import org.scalatest.FunSuite + +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkException +import org.apache.spark.SharedSparkContext + +/* A trivial (but unserializable) container for trivial functions */ +class UnserializableClass { + def op[T](x: T) = x.toString + + def pred[T](x: T) = x.toString.length % 2 == 0 +} + +class ProactiveClosureSerializationSuite extends FunSuite with SharedSparkContext { + + def fixture = (sc.parallelize(0 until 1000).map(_.toString), new UnserializableClass) + + test("throws expected serialization exceptions on actions") { + val (data, uc) = fixture + + val ex = intercept[SparkException] { + data.map(uc.op(_)).count + } + + assert(ex.getMessage.matches(".*Task not serializable.*")) + } + + // There is probably a cleaner way to eliminate boilerplate here, but we're + // iterating over a map from transformation names to functions that perform that + // transformation on a given RDD, creating one test case for each + + for (transformation <- + Map("map" -> map _, "flatMap" -> flatMap _, "filter" -> filter _, "mapWith" -> mapWith _, + "mapPartitions" -> mapPartitions _, "mapPartitionsWithIndex" -> mapPartitionsWithIndex _, + "mapPartitionsWithContext" -> mapPartitionsWithContext _, "filterWith" -> filterWith _)) { + val (name, xf) = transformation + + test(s"$name transformations throw proactive serialization exceptions") { + val (data, uc) = fixture + + val ex = intercept[SparkException] { + xf(data, uc) + } + + assert(ex.getMessage.matches(".*Task not serializable.*"), s"RDD.$name doesn't proactively throw NotSerializableException") + } + } + + def map(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.map(y => uc.op(y)) + + def mapWith(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.mapWith(x => x.toString)((x,y) => x + uc.op(y)) + + def flatMap(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.flatMap(y=>Seq(uc.op(y))) + + def filter(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.filter(y=>uc.pred(y)) + + def filterWith(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.filterWith(x => x.toString)((x,y) => uc.pred(y)) + + def mapPartitions(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.mapPartitions(_.map(y => uc.op(y))) + + def mapPartitionsWithIndex(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.mapPartitionsWithIndex((_, it) => it.map(y => uc.op(y))) + + def mapPartitionsWithContext(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.mapPartitionsWithContext((_, it) => it.map(y => uc.op(y))) + +} diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index 439e5644e20a3..c635da6cacd70 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -50,6 +50,27 @@ class ClosureCleanerSuite extends FunSuite { val obj = new TestClassWithNesting(1) assert(obj.run() === 96) // 4 * (1+2+3+4) + 4 * (1+2+3+4) + 16 * 1 } + + test("capturing free variables in closures at RDD definition") { + val obj = new TestCaptureVarClass() + val (ones, onesPlusZeroes) = obj.run() + + assert(ones === onesPlusZeroes) + } + + test("capturing free variable fields in closures at RDD definition") { + val obj = new TestCaptureFieldClass() + val (ones, onesPlusZeroes) = obj.run() + + assert(ones === onesPlusZeroes) + } + + test("capturing arrays in closures at RDD definition") { + val obj = new TestCaptureArrayEltClass() + val (observed, expected) = obj.run() + + assert(observed === expected) + } } // A non-serializable class we create in closures to make sure that we aren't @@ -143,3 +164,50 @@ class TestClassWithNesting(val y: Int) extends Serializable { } } } + +class TestCaptureFieldClass extends Serializable { + class ZeroBox extends Serializable { + var zero = 0 + } + + def run(): (Int, Int) = { + val zb = new ZeroBox + + withSpark(new SparkContext("local", "test")) {sc => + val ones = sc.parallelize(Array(1, 1, 1, 1, 1)) + val onesPlusZeroes = ones.map(_ + zb.zero) + + zb.zero = 5 + + (ones.reduce(_ + _), onesPlusZeroes.reduce(_ + _)) + } + } +} + +class TestCaptureArrayEltClass extends Serializable { + def run(): (Int, Int) = { + withSpark(new SparkContext("local", "test")) {sc => + val rdd = sc.parallelize(1 to 10) + val data = Array(1, 2, 3) + val expected = data(0) + val mapped = rdd.map(x => data(0)) + data(0) = 4 + (mapped.first, expected) + } + } +} + +class TestCaptureVarClass extends Serializable { + def run(): (Int, Int) = { + var zero = 0 + + withSpark(new SparkContext("local", "test")) {sc => + val ones = sc.parallelize(Array(1, 1, 1, 1, 1)) + val onesPlusZeroes = ones.map(_ + zero) + + zero = 5 + + (ones.reduce(_ + _), onesPlusZeroes.reduce(_ + _)) + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index 28d34dd9a1a41..c65e36636fe10 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -62,7 +62,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert( graph.edges.count() === rawEdges.size ) // Vertices not explicitly provided but referenced by edges should be created automatically assert( graph.vertices.count() === 100) - graph.triplets.map { et => + graph.triplets.collect.map { et => assert((et.srcId < 10 && et.srcAttr) || (et.srcId >= 10 && !et.srcAttr)) assert((et.dstId < 10 && et.dstAttr) || (et.dstId >= 10 && !et.dstAttr)) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index d043200f71a0b..4759b629a9931 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -539,7 +539,7 @@ abstract class DStream[T: ClassTag] ( * on each RDD of 'this' DStream. */ def transform[U: ClassTag](transformFunc: RDD[T] => RDD[U]): DStream[U] = { - transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r))) + transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r), false)) } /** @@ -547,7 +547,7 @@ abstract class DStream[T: ClassTag] ( * on each RDD of 'this' DStream. */ def transform[U: ClassTag](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = { - val cleanedF = context.sparkContext.clean(transformFunc) + val cleanedF = context.sparkContext.clean(transformFunc, false) val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => { assert(rdds.length == 1) cleanedF(rdds.head.asInstanceOf[RDD[T]], time) @@ -562,7 +562,7 @@ abstract class DStream[T: ClassTag] ( def transformWith[U: ClassTag, V: ClassTag]( other: DStream[U], transformFunc: (RDD[T], RDD[U]) => RDD[V] ): DStream[V] = { - val cleanedF = ssc.sparkContext.clean(transformFunc) + val cleanedF = ssc.sparkContext.clean(transformFunc, false) transformWith(other, (rdd1: RDD[T], rdd2: RDD[U], time: Time) => cleanedF(rdd1, rdd2)) } @@ -573,7 +573,7 @@ abstract class DStream[T: ClassTag] ( def transformWith[U: ClassTag, V: ClassTag]( other: DStream[U], transformFunc: (RDD[T], RDD[U], Time) => RDD[V] ): DStream[V] = { - val cleanedF = ssc.sparkContext.clean(transformFunc) + val cleanedF = ssc.sparkContext.clean(transformFunc, false) val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => { assert(rdds.length == 2) val rdd1 = rdds(0).asInstanceOf[RDD[T]] From e55cc4bae52a3de728939244780abc662713b768 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Thu, 10 Apr 2014 00:37:21 -0700 Subject: [PATCH 041/641] SPARK-1446: Spark examples should not do a System.exit Spark examples should exit nice using SparkContext.stop() method, rather than System.exit System.exit can cause issues like in SPARK-1407 Author: Sandeep Closes #370 from techaddict/1446 and squashes the following commits: e9234cf [Sandeep] SPARK-1446: Spark examples should not do a System.exit Spark examples should exit nice using SparkContext.stop() method, rather than System.exit System.exit can cause issues like in SPARK-1407 --- .../org/apache/spark/examples/JavaHdfsLR.java | 2 +- .../apache/spark/examples/JavaLogQuery.java | 2 +- .../apache/spark/examples/JavaPageRank.java | 2 +- .../org/apache/spark/examples/JavaTC.java | 2 +- .../apache/spark/examples/JavaWordCount.java | 6 +++--- .../apache/spark/mllib/examples/JavaALS.java | 2 +- .../spark/mllib/examples/JavaKMeans.java | 2 +- .../apache/spark/mllib/examples/JavaLR.java | 2 +- .../apache/spark/examples/BroadcastTest.scala | 2 +- .../spark/examples/CassandraCQLTest.scala | 2 +- .../examples/ExceptionHandlingTest.scala | 2 +- .../apache/spark/examples/GroupByTest.scala | 5 ++--- .../org/apache/spark/examples/HBaseTest.scala | 6 +++--- .../org/apache/spark/examples/HdfsTest.scala | 2 +- .../org/apache/spark/examples/LocalALS.scala | 2 +- .../spark/examples/MultiBroadcastTest.scala | 2 +- .../examples/SimpleSkewedGroupByTest.scala | 7 +++---- .../spark/examples/SkewedGroupByTest.scala | 7 +++---- .../org/apache/spark/examples/SparkALS.scala | 4 ++-- .../apache/spark/examples/SparkHdfsLR.scala | 2 +- .../apache/spark/examples/SparkKMeans.scala | 20 +++++++++---------- .../org/apache/spark/examples/SparkLR.scala | 2 +- .../apache/spark/examples/SparkPageRank.scala | 3 +-- .../org/apache/spark/examples/SparkTC.scala | 2 +- .../spark/examples/SparkTachyonHdfsLR.scala | 2 +- .../spark/examples/SparkTachyonPi.scala | 6 +++--- .../bagel/WikipediaPageRankStandalone.scala | 2 +- .../spark/examples/mllib/TallSkinnyPCA.scala | 2 +- .../spark/examples/mllib/TallSkinnySVD.scala | 2 +- .../streaming/examples/HdfsWordCount.scala | 1 - .../streaming/examples/KafkaWordCount.scala | 5 ++--- .../streaming/examples/MQTTWordCount.scala | 2 +- .../streaming/examples/QueueStream.scala | 11 +++++----- .../examples/StatefulNetworkWordCount.scala | 2 +- .../streaming/examples/ZeroMQWordCount.scala | 2 +- 35 files changed, 60 insertions(+), 67 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index 6b49244ba459d..bd96274021756 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -138,6 +138,6 @@ public static void main(String[] args) { System.out.print("Final w: "); printWeights(w); - System.exit(0); + sc.stop(); } } 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 617e4a6d045e0..2a4278d3c30e5 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -126,6 +126,6 @@ public Stats call(Stats stats, Stats stats2) { for (Tuple2 t : output) { System.out.println(t._1() + "\t" + t._2()); } - System.exit(0); + jsc.stop(); } } 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 8513ba07e7705..e31f676f5fd4c 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -117,6 +117,6 @@ public Double call(Double sum) { System.out.println(tuple._1() + " has rank: " + tuple._2() + "."); } - System.exit(0); + ctx.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index 6cfe25c80ecc6..1d776940f06c6 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -96,6 +96,6 @@ public Tuple2 call(Tuple2 e) { } while (nextCount != oldCount); System.out.println("TC has " + tc.count() + " edges."); - System.exit(0); + sc.stop(); } } 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 3ae1d8f7ca938..87c1b80981961 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -48,14 +48,14 @@ public Iterable call(String s) { return Arrays.asList(SPACE.split(s)); } }); - + JavaPairRDD ones = words.mapToPair(new PairFunction() { @Override public Tuple2 call(String s) { return new Tuple2(s, 1); } }); - + JavaPairRDD counts = ones.reduceByKey(new Function2() { @Override public Integer call(Integer i1, Integer i2) { @@ -67,6 +67,6 @@ public Integer call(Integer i1, Integer i2) { for (Tuple2 tuple : output) { System.out.println(tuple._1() + ": " + tuple._2()); } - System.exit(0); + ctx.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java index 64a3a04fb7296..c516199d61c72 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java @@ -85,6 +85,6 @@ public static void main(String[] args) { outputDir + "/productFeatures"); System.out.println("Final user/product features written to " + outputDir); - System.exit(0); + sc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java index 7b0ec36424e97..7461609ab9e8f 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java @@ -79,6 +79,6 @@ public static void main(String[] args) { double cost = model.computeCost(points.rdd()); System.out.println("Cost: " + cost); - System.exit(0); + sc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java index cd8879ff886e2..e3ab87cc722f3 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java @@ -77,6 +77,6 @@ public static void main(String[] args) { System.out.print("Final w: " + model.weights()); - System.exit(0); + sc.stop(); } } diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index 4d2f45df85fc6..c8c916bb45e00 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -56,6 +56,6 @@ object BroadcastTest { println("Iteration %d took %.0f milliseconds".format(i, (System.nanoTime - startTime) / 1E6)) } - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala index ee283ce6abac2..1f8d7cb5995b8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala @@ -58,7 +58,7 @@ import org.apache.spark.SparkContext._ prod_id, quantity) VALUES ('charlie', 1385983649000, 'iphone', 2); */ - + /** * This example demonstrates how to read and write to cassandra column family created using CQL3 * using Spark. diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala index fdb976dfc6aba..be7d39549a28d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala @@ -34,6 +34,6 @@ object ExceptionHandlingTest { } } - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index 36534e59353cd..29114c6dabcdb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -28,7 +28,7 @@ object GroupByTest { "Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers]") System.exit(1) } - + var numMappers = if (args.length > 1) args(1).toInt else 2 var numKVPairs = if (args.length > 2) args(2).toInt else 1000 var valSize = if (args.length > 3) args(3).toInt else 1000 @@ -52,7 +52,6 @@ object GroupByTest { println(pairs1.groupByKey(numReducers).count) - System.exit(0) + sc.stop() } } - diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala index 65d67356be2f6..700121d16dd60 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala @@ -30,7 +30,7 @@ object HBaseTest { val conf = HBaseConfiguration.create() - // Other options for configuring scan behavior are available. More information available at + // Other options for configuring scan behavior are available. More information available at // http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html conf.set(TableInputFormat.INPUT_TABLE, args(1)) @@ -41,12 +41,12 @@ object HBaseTest { admin.createTable(tableDesc) } - val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat], + val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat], classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], classOf[org.apache.hadoop.hbase.client.Result]) hBaseRDD.count() - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index c3597d94a224e..dd6d5205133be 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -32,6 +32,6 @@ object HdfsTest { val end = System.currentTimeMillis() println("Iteration " + iter + " took " + (end-start) + " ms") } - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index 0095cb8425456..37ad4bd0999bd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -120,7 +120,7 @@ object LocalALS { } } printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) - + val R = generateR() // Initialize m and u randomly diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index 4aef04fc060b6..97321ab8f41db 100644 --- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -51,6 +51,6 @@ object MultiBroadcastTest { // Collect the small RDD so we can print the observed sizes locally. observedSizes.collect().foreach(i => println(i)) - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 1fdb324b89f3a..d05eedd31caa0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -27,7 +27,7 @@ object SimpleSkewedGroupByTest { System.err.println("Usage: SimpleSkewedGroupByTest " + "[numMappers] [numKVPairs] [valSize] [numReducers] [ratio]") System.exit(1) - } + } var numMappers = if (args.length > 1) args(1).toInt else 2 var numKVPairs = if (args.length > 2) args(2).toInt else 1000 @@ -58,14 +58,13 @@ object SimpleSkewedGroupByTest { }.cache // Enforce that everything has been calculated and in cache pairs1.count - + println("RESULT: " + pairs1.groupByKey(numReducers).count) // Print how many keys each reducer got (for debugging) // println("RESULT: " + pairs1.groupByKey(numReducers) // .map{case (k,v) => (k, v.size)} // .collectAsMap) - System.exit(0) + sc.stop() } } - diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 966478fe4a258..fd9f043247d18 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -27,7 +27,7 @@ object SkewedGroupByTest { System.err.println( "Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers]") System.exit(1) - } + } var numMappers = if (args.length > 1) args(1).toInt else 2 var numKVPairs = if (args.length > 2) args(2).toInt else 1000 @@ -53,10 +53,9 @@ object SkewedGroupByTest { }.cache() // Enforce that everything has been calculated and in cache pairs1.count() - + println(pairs1.groupByKey(numReducers).count()) - System.exit(0) + sc.stop() } } - 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 f59ab7e7cc24a..68f151a2c47fe 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -112,7 +112,7 @@ object SparkALS { val sc = new SparkContext(host, "SparkALS", System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) - + val R = generateR() // Initialize m and u randomly @@ -137,6 +137,6 @@ object SparkALS { println() } - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 038afbcba80a3..d8de8745c15d9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -52,7 +52,7 @@ object SparkHdfsLR { val inputPath = args(1) val conf = SparkHadoopUtil.get.newConfiguration() val sc = new SparkContext(args(0), "SparkHdfsLR", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), InputFormatInfo.computePreferredLocations( Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) )) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index 9fe24652358f3..1a8b21618e23a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -28,16 +28,16 @@ import org.apache.spark.SparkContext._ object SparkKMeans { val R = 1000 // Scaling factor val rand = new Random(42) - + def parseVector(line: String): Vector = { new Vector(line.split(' ').map(_.toDouble)) } - + def closestPoint(p: Vector, centers: Array[Vector]): Int = { var index = 0 var bestIndex = 0 var closest = Double.PositiveInfinity - + for (i <- 0 until centers.length) { val tempDist = p.squaredDist(centers(i)) if (tempDist < closest) { @@ -45,7 +45,7 @@ object SparkKMeans { bestIndex = i } } - + bestIndex } @@ -60,22 +60,22 @@ object SparkKMeans { val data = lines.map(parseVector _).cache() val K = args(2).toInt val convergeDist = args(3).toDouble - + val kPoints = data.takeSample(withReplacement = false, K, 42).toArray var tempDist = 1.0 while(tempDist > convergeDist) { val closest = data.map (p => (closestPoint(p, kPoints), (p, 1))) - + val pointStats = closest.reduceByKey{case ((x1, y1), (x2, y2)) => (x1 + x2, y1 + y2)} - + val newPoints = pointStats.map {pair => (pair._1, pair._2._1 / pair._2._2)}.collectAsMap() - + tempDist = 0.0 for (i <- 0 until K) { tempDist += kPoints(i).squaredDist(newPoints(i)) } - + for (newP <- newPoints) { kPoints(newP._1) = newP._2 } @@ -84,6 +84,6 @@ object SparkKMeans { println("Final centers:") kPoints.foreach(println) - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index c54a55bdb4a11..3a2699d4d996b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -66,6 +66,6 @@ object SparkLR { } println("Final w: " + w) - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index d203f4d20e15f..45b6e10f3ea9e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -57,7 +57,6 @@ object SparkPageRank { val output = ranks.collect() output.foreach(tup => println(tup._1 + " has rank: " + tup._2 + ".")) - System.exit(0) + ctx.stop() } } - diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index 24e8afa26bc5f..eb47cf027cb10 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -70,6 +70,6 @@ object SparkTC { } while (nextCount != oldCount) println("TC has " + tc.count() + " edges.") - System.exit(0) + spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala index 53b303d658386..5698d4746495d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala @@ -75,6 +75,6 @@ object SparkTachyonHdfsLR { } println("Final w: " + w) - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala index ce78f0876ed7c..2b207fd8d3e16 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala @@ -34,10 +34,10 @@ object SparkTachyonPi { } val spark = new SparkContext(args(0), "SparkTachyonPi", System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) - + val slices = if (args.length > 1) args(1).toInt else 2 val n = 100000 * slices - + val rdd = spark.parallelize(1 to n, slices) rdd.persist(StorageLevel.OFF_HEAP) val count = rdd.map { i => @@ -46,7 +46,7 @@ object SparkTachyonPi { if (x * x + y * y < 1) 1 else 0 }.reduce(_ + _) println("Pi is roughly " + 4.0 * count / n) - + spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index 7aac6a13597e6..dee3cb6c0abae 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -79,7 +79,7 @@ object WikipediaPageRankStandalone { val time = (System.currentTimeMillis - startTime) / 1000.0 println("Completed %d iterations in %f seconds: %f seconds per iteration" .format(numIterations, time, time / numIterations)) - System.exit(0) + sc.stop() } def parseArticle(line: String): (String, Array[String]) = { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala index a177435e606ab..61b9655cd3759 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.examples.mllib - + import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.linalg.Vectors diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala index 49d09692c8e4a..9aeebf58eabfb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.examples.mllib - + import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.linalg.Vectors diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala index 954bcc9b6ef5d..1c0ce3111e290 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala @@ -53,4 +53,3 @@ object HdfsWordCount { ssc.awaitTermination() } } - diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index 6bccd1d88401a..cca0be2cbb9c9 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -61,7 +61,7 @@ object KafkaWordCount { val wordCounts = words.map(x => (x, 1L)) .reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2) wordCounts.print() - + ssc.start() ssc.awaitTermination() } @@ -83,7 +83,7 @@ object KafkaWordCountProducer { val props = new Properties() props.put("metadata.broker.list", brokers) props.put("serializer.class", "kafka.serializer.StringEncoder") - + val config = new ProducerConfig(props) val producer = new Producer[String, String](config) @@ -102,4 +102,3 @@ object KafkaWordCountProducer { } } - diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index 0a68ac84c2424..656222e0c1b31 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -26,7 +26,7 @@ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.mqtt._ /** - * A simple Mqtt publisher for demonstration purposes, repeatedly publishes + * A simple Mqtt publisher for demonstration purposes, repeatedly publishes * Space separated String Message "hello mqtt demo for spark streaming" */ object MQTTPublisher { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala index 4d4968ba6ae3e..612ecf7b7821a 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala @@ -24,7 +24,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ object QueueStream { - + def main(args: Array[String]) { if (args.length < 1) { System.err.println("Usage: QueueStream ") @@ -37,23 +37,22 @@ object QueueStream { val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) - // Create the queue through which RDDs can be pushed to + // Create the queue through which RDDs can be pushed to // a QueueInputDStream val rddQueue = new SynchronizedQueue[RDD[Int]]() - + // Create the QueueInputDStream and use it do some processing val inputStream = ssc.queueStream(rddQueue) val mappedStream = inputStream.map(x => (x % 10, 1)) val reducedStream = mappedStream.reduceByKey(_ + _) - reducedStream.print() + reducedStream.print() ssc.start() - + // Create and push some RDDs into for (i <- 1 to 30) { rddQueue += ssc.sparkContext.makeRDD(1 to 1000, 10) Thread.sleep(1000) } ssc.stop() - System.exit(0) } } diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala index c2d84a8e0861e..14f65a2f8d46c 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -58,7 +58,7 @@ object StatefulNetworkWordCount { ssc.checkpoint(".") // Create a NetworkInputDStream on target ip:port and count the - // words in input stream of \n delimited test (eg. generated by 'nc') + // words in input stream of \n delimited test (eg. generated by 'nc') val lines = ssc.socketTextStream(args(1), args(2).toInt) val words = lines.flatMap(_.split(" ")) val wordDstream = words.map(x => (x, 1)) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index 35f8f885f8f0e..445d2028582af 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -60,7 +60,7 @@ object SimpleZeroMQPublisher { * To work with zeroMQ, some native libraries have to be installed. * Install zeroMQ (release 2.1) core libraries. [ZeroMQ Install guide] * (http://www.zeromq.org/intro:get-the-software) - * + * * Usage: ZeroMQWordCount * In local mode, should be 'local[n]' with n > 1 * and describe where zeroMq publisher is running. From e6d4a74d2d92345985c1603f9b526a6347adb7cf Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 10 Apr 2014 02:10:40 -0700 Subject: [PATCH 042/641] Revert "SPARK-729: Closures not always serialized at capture time" This reverts commit 8ca3b2bc90a63b23a03f339e390174cd7a672b40. --- .../scala/org/apache/spark/SparkContext.scala | 16 +--- .../main/scala/org/apache/spark/rdd/RDD.scala | 6 +- .../apache/spark/util/ClosureCleaner.scala | 21 +---- .../scala/org/apache/spark/FailureSuite.scala | 17 +--- .../ProactiveClosureSerializationSuite.scala | 94 ------------------- .../spark/util/ClosureCleanerSuite.scala | 68 -------------- .../org/apache/spark/graphx/GraphSuite.scala | 2 +- .../spark/streaming/dstream/DStream.scala | 8 +- 8 files changed, 14 insertions(+), 218 deletions(-) delete mode 100644 core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 545807ffbce55..76305237b03d5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1002,9 +1002,7 @@ class SparkContext(config: SparkConf) extends Logging { require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p") } val callSite = getCallSite - // There's no need to check this function for serializability, - // since it will be run right away. - val cleanedFunc = clean(func, false) + val cleanedFunc = clean(func) logInfo("Starting job: " + callSite) val start = System.nanoTime dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal, @@ -1137,18 +1135,14 @@ class SparkContext(config: SparkConf) extends Logging { def cancelAllJobs() { dagScheduler.cancelAllJobs() } - + /** * Clean a closure to make it ready to serialized and send to tasks * (removes unreferenced variables in $outer's, updates REPL variables) - * - * @param f closure to be cleaned and optionally serialized - * @param captureNow whether or not to serialize this closure and capture any free - * variables immediately; defaults to true. If this is set and f is not serializable, - * it will raise an exception. */ - private[spark] def clean[F <: AnyRef : ClassTag](f: F, captureNow: Boolean = true): F = { - ClosureCleaner.clean(f, captureNow) + private[spark] def clean[F <: AnyRef](f: F): F = { + ClosureCleaner.clean(f) + f } /** 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 e363ea777d8eb..3437b2cac19c2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -660,16 +660,14 @@ abstract class RDD[T: ClassTag]( * Applies a function f to all elements of this RDD. */ def foreach(f: T => Unit) { - val cleanF = sc.clean(f) - sc.runJob(this, (iter: Iterator[T]) => iter.foreach(cleanF)) + sc.runJob(this, (iter: Iterator[T]) => iter.foreach(f)) } /** * Applies a function f to each partition of this RDD. */ def foreachPartition(f: Iterator[T] => Unit) { - val cleanF = sc.clean(f) - sc.runJob(this, (iter: Iterator[T]) => cleanF(iter)) + sc.runJob(this, (iter: Iterator[T]) => f(iter)) } /** diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index e474b1a850d65..cdbbc65292188 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -22,14 +22,10 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.collection.mutable.Map import scala.collection.mutable.Set -import scala.reflect.ClassTag - import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ import org.apache.spark.Logging -import org.apache.spark.SparkEnv -import org.apache.spark.SparkException private[spark] object ClosureCleaner extends Logging { // Get an ASM class reader for a given class from the JAR that loaded it @@ -105,7 +101,7 @@ private[spark] object ClosureCleaner extends Logging { } } - def clean[F <: AnyRef : ClassTag](func: F, captureNow: Boolean = true): F = { + def clean(func: AnyRef) { // TODO: cache outerClasses / innerClasses / accessedFields val outerClasses = getOuterClasses(func) val innerClasses = getInnerClasses(func) @@ -154,21 +150,6 @@ private[spark] object ClosureCleaner extends Logging { field.setAccessible(true) field.set(func, outer) } - - if (captureNow) { - cloneViaSerializing(func) - } else { - func - } - } - - private def cloneViaSerializing[T: ClassTag](func: T): T = { - try { - val serializer = SparkEnv.get.closureSerializer.newInstance() - serializer.deserialize[T](serializer.serialize[T](func)) - } catch { - case ex: Exception => throw new SparkException("Task not serializable: " + ex.toString) - } } private def instantiateClass(cls: Class[_], outer: AnyRef, inInterpreter: Boolean): AnyRef = { diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index 4f9300419e6f8..12dbebcb28644 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -107,7 +107,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { FailureSuiteState.clear() } - test("failure because closure in final-stage task is not serializable") { + test("failure because task closure is not serializable") { sc = new SparkContext("local[1,1]", "test") val a = new NonSerializable @@ -118,13 +118,6 @@ class FailureSuite extends FunSuite with LocalSparkContext { assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) - FailureSuiteState.clear() - } - - test("failure because closure in early-stage task is not serializable") { - sc = new SparkContext("local[1,1]", "test") - val a = new NonSerializable - // Non-serializable closure in an earlier stage val thrown1 = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() @@ -132,13 +125,6 @@ class FailureSuite extends FunSuite with LocalSparkContext { assert(thrown1.getClass === classOf[SparkException]) assert(thrown1.getMessage.contains("NotSerializableException")) - FailureSuiteState.clear() - } - - test("failure because closure in foreach task is not serializable") { - sc = new SparkContext("local[1,1]", "test") - val a = new NonSerializable - // Non-serializable closure in foreach function val thrown2 = intercept[SparkException] { sc.parallelize(1 to 10, 2).foreach(x => println(a)) @@ -149,6 +135,5 @@ class FailureSuite extends FunSuite with LocalSparkContext { FailureSuiteState.clear() } - // TODO: Need to add tests with shuffle fetch failures. } diff --git a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala deleted file mode 100644 index 76662264e7e94..0000000000000 --- a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala +++ /dev/null @@ -1,94 +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.io.NotSerializableException - -import org.scalatest.FunSuite - -import org.apache.spark.rdd.RDD -import org.apache.spark.SparkException -import org.apache.spark.SharedSparkContext - -/* A trivial (but unserializable) container for trivial functions */ -class UnserializableClass { - def op[T](x: T) = x.toString - - def pred[T](x: T) = x.toString.length % 2 == 0 -} - -class ProactiveClosureSerializationSuite extends FunSuite with SharedSparkContext { - - def fixture = (sc.parallelize(0 until 1000).map(_.toString), new UnserializableClass) - - test("throws expected serialization exceptions on actions") { - val (data, uc) = fixture - - val ex = intercept[SparkException] { - data.map(uc.op(_)).count - } - - assert(ex.getMessage.matches(".*Task not serializable.*")) - } - - // There is probably a cleaner way to eliminate boilerplate here, but we're - // iterating over a map from transformation names to functions that perform that - // transformation on a given RDD, creating one test case for each - - for (transformation <- - Map("map" -> map _, "flatMap" -> flatMap _, "filter" -> filter _, "mapWith" -> mapWith _, - "mapPartitions" -> mapPartitions _, "mapPartitionsWithIndex" -> mapPartitionsWithIndex _, - "mapPartitionsWithContext" -> mapPartitionsWithContext _, "filterWith" -> filterWith _)) { - val (name, xf) = transformation - - test(s"$name transformations throw proactive serialization exceptions") { - val (data, uc) = fixture - - val ex = intercept[SparkException] { - xf(data, uc) - } - - assert(ex.getMessage.matches(".*Task not serializable.*"), s"RDD.$name doesn't proactively throw NotSerializableException") - } - } - - def map(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.map(y => uc.op(y)) - - def mapWith(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.mapWith(x => x.toString)((x,y) => x + uc.op(y)) - - def flatMap(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.flatMap(y=>Seq(uc.op(y))) - - def filter(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.filter(y=>uc.pred(y)) - - def filterWith(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.filterWith(x => x.toString)((x,y) => uc.pred(y)) - - def mapPartitions(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.mapPartitions(_.map(y => uc.op(y))) - - def mapPartitionsWithIndex(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.mapPartitionsWithIndex((_, it) => it.map(y => uc.op(y))) - - def mapPartitionsWithContext(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.mapPartitionsWithContext((_, it) => it.map(y => uc.op(y))) - -} diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index c635da6cacd70..439e5644e20a3 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -50,27 +50,6 @@ class ClosureCleanerSuite extends FunSuite { val obj = new TestClassWithNesting(1) assert(obj.run() === 96) // 4 * (1+2+3+4) + 4 * (1+2+3+4) + 16 * 1 } - - test("capturing free variables in closures at RDD definition") { - val obj = new TestCaptureVarClass() - val (ones, onesPlusZeroes) = obj.run() - - assert(ones === onesPlusZeroes) - } - - test("capturing free variable fields in closures at RDD definition") { - val obj = new TestCaptureFieldClass() - val (ones, onesPlusZeroes) = obj.run() - - assert(ones === onesPlusZeroes) - } - - test("capturing arrays in closures at RDD definition") { - val obj = new TestCaptureArrayEltClass() - val (observed, expected) = obj.run() - - assert(observed === expected) - } } // A non-serializable class we create in closures to make sure that we aren't @@ -164,50 +143,3 @@ class TestClassWithNesting(val y: Int) extends Serializable { } } } - -class TestCaptureFieldClass extends Serializable { - class ZeroBox extends Serializable { - var zero = 0 - } - - def run(): (Int, Int) = { - val zb = new ZeroBox - - withSpark(new SparkContext("local", "test")) {sc => - val ones = sc.parallelize(Array(1, 1, 1, 1, 1)) - val onesPlusZeroes = ones.map(_ + zb.zero) - - zb.zero = 5 - - (ones.reduce(_ + _), onesPlusZeroes.reduce(_ + _)) - } - } -} - -class TestCaptureArrayEltClass extends Serializable { - def run(): (Int, Int) = { - withSpark(new SparkContext("local", "test")) {sc => - val rdd = sc.parallelize(1 to 10) - val data = Array(1, 2, 3) - val expected = data(0) - val mapped = rdd.map(x => data(0)) - data(0) = 4 - (mapped.first, expected) - } - } -} - -class TestCaptureVarClass extends Serializable { - def run(): (Int, Int) = { - var zero = 0 - - withSpark(new SparkContext("local", "test")) {sc => - val ones = sc.parallelize(Array(1, 1, 1, 1, 1)) - val onesPlusZeroes = ones.map(_ + zero) - - zero = 5 - - (ones.reduce(_ + _), onesPlusZeroes.reduce(_ + _)) - } - } -} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index c65e36636fe10..28d34dd9a1a41 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -62,7 +62,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert( graph.edges.count() === rawEdges.size ) // Vertices not explicitly provided but referenced by edges should be created automatically assert( graph.vertices.count() === 100) - graph.triplets.collect.map { et => + graph.triplets.map { et => assert((et.srcId < 10 && et.srcAttr) || (et.srcId >= 10 && !et.srcAttr)) assert((et.dstId < 10 && et.dstAttr) || (et.dstId >= 10 && !et.dstAttr)) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 4759b629a9931..d043200f71a0b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -539,7 +539,7 @@ abstract class DStream[T: ClassTag] ( * on each RDD of 'this' DStream. */ def transform[U: ClassTag](transformFunc: RDD[T] => RDD[U]): DStream[U] = { - transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r), false)) + transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r))) } /** @@ -547,7 +547,7 @@ abstract class DStream[T: ClassTag] ( * on each RDD of 'this' DStream. */ def transform[U: ClassTag](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = { - val cleanedF = context.sparkContext.clean(transformFunc, false) + val cleanedF = context.sparkContext.clean(transformFunc) val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => { assert(rdds.length == 1) cleanedF(rdds.head.asInstanceOf[RDD[T]], time) @@ -562,7 +562,7 @@ abstract class DStream[T: ClassTag] ( def transformWith[U: ClassTag, V: ClassTag]( other: DStream[U], transformFunc: (RDD[T], RDD[U]) => RDD[V] ): DStream[V] = { - val cleanedF = ssc.sparkContext.clean(transformFunc, false) + val cleanedF = ssc.sparkContext.clean(transformFunc) transformWith(other, (rdd1: RDD[T], rdd2: RDD[U], time: Time) => cleanedF(rdd1, rdd2)) } @@ -573,7 +573,7 @@ abstract class DStream[T: ClassTag] ( def transformWith[U: ClassTag, V: ClassTag]( other: DStream[U], transformFunc: (RDD[T], RDD[U], Time) => RDD[V] ): DStream[V] = { - val cleanedF = ssc.sparkContext.clean(transformFunc, false) + val cleanedF = ssc.sparkContext.clean(transformFunc) val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => { assert(rdds.length == 2) val rdd1 = rdds(0).asInstanceOf[RDD[T]] From a74fbbbca8f0d89b2e0e4e8751a93d33efc4fa9e Mon Sep 17 00:00:00 2001 From: witgo Date: Thu, 10 Apr 2014 10:35:24 -0700 Subject: [PATCH 043/641] Fix SPARK-1413: Parquet messes up stdout and stdin when used in Spark REPL Author: witgo Closes #325 from witgo/SPARK-1413 and squashes the following commits: e57cd8e [witgo] use scala reflection to access and call the SLF4JBridgeHandler methods 45c8f40 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 5e35d87 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 0d5f819 [witgo] review commit 45e5b70 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 fa69dcf [witgo] Merge branch 'master' into SPARK-1413 3c98dc4 [witgo] Merge branch 'master' into SPARK-1413 38160cb [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 ba09bcd [witgo] remove set the parquet log level a63d574 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 5231ecd [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 3feb635 [witgo] parquet logger use parent handler fa00d5d [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 8bb6ffd [witgo] enableLogForwarding note fix edd9630 [witgo] move to f447f50 [witgo] merging master 5ad52bd [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 76670c1 [witgo] review commit 70f3c64 [witgo] Fix SPARK-1413 --- .../main/scala/org/apache/spark/Logging.scala | 20 +++++++++--- .../spark/sql/parquet/ParquetRelation.scala | 31 +++++-------------- 2 files changed, 23 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index e5e15617acb10..9d429dceeb858 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -28,7 +28,7 @@ import org.apache.spark.annotation.DeveloperApi * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows * logging messages at different levels using methods that only evaluate parameters lazily if the * log level is enabled. - * + * * NOTE: DO NOT USE this class outside of Spark. It is intended as an internal utility. * This will likely be changed or removed in future releases. */ @@ -60,7 +60,7 @@ trait Logging { protected def logDebug(msg: => String) { if (log.isDebugEnabled) log.debug(msg) } - + protected def logTrace(msg: => String) { if (log.isTraceEnabled) log.trace(msg) } @@ -117,10 +117,10 @@ trait Logging { val defaultLogProps = "org/apache/spark/log4j-defaults.properties" val classLoader = this.getClass.getClassLoader Option(classLoader.getResource(defaultLogProps)) match { - case Some(url) => + case Some(url) => PropertyConfigurator.configure(url) log.info(s"Using Spark's default log4j profile: $defaultLogProps") - case None => + case None => System.err.println(s"Spark was unable to load $defaultLogProps") } } @@ -135,4 +135,16 @@ trait Logging { private object Logging { @volatile private var initialized = false val initLock = new Object() + try { + // We use reflection here to handle the case where users remove the + // slf4j-to-jul bridge order to route their logs to JUL. + val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler") + bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null) + val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean] + if (!installed) { + bridgeClass.getMethod("install").invoke(null) + } + } catch { + case e: ClassNotFoundException => // can't log anything yet so just fail silently + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 505ad0a2c77c1..4d7c86a3a4fc7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -82,30 +82,13 @@ private[sql] case class ParquetRelation(val path: String) private[sql] object ParquetRelation { def enableLogForwarding() { - // Note: Parquet does not use forwarding to parent loggers which - // is required for the JUL-SLF4J bridge to work. Also there is - // a default logger that appends to Console which needs to be - // reset. - import org.slf4j.bridge.SLF4JBridgeHandler - import java.util.logging.Logger - import java.util.logging.LogManager - - val loggerNames = Seq( - "parquet.hadoop.ColumnChunkPageWriteStore", - "parquet.hadoop.InternalParquetRecordWriter", - "parquet.hadoop.ParquetRecordReader", - "parquet.hadoop.ParquetInputFormat", - "parquet.hadoop.ParquetOutputFormat", - "parquet.hadoop.ParquetFileReader", - "parquet.hadoop.InternalParquetRecordReader", - "parquet.hadoop.codec.CodecConfig") - LogManager.getLogManager.reset() - SLF4JBridgeHandler.install() - for(name <- loggerNames) { - val logger = Logger.getLogger(name) - logger.setParent(Logger.getLogger(Logger.GLOBAL_LOGGER_NAME)) - logger.setUseParentHandlers(true) - } + // Note: Logger.getLogger("parquet") has a default logger + // that appends to Console which needs to be cleared. + val parquetLogger = java.util.logging.Logger.getLogger("parquet") + parquetLogger.getHandlers.foreach(parquetLogger.removeHandler) + // TODO(witgo): Need to set the log level ? + // if(parquetLogger.getLevel != null) parquetLogger.setLevel(null) + if (!parquetLogger.getUseParentHandlers) parquetLogger.setUseParentHandlers(true) } // The element type for the RDDs that this relation maps to. From 79820fe825ed7c09d55f50503b7ab53d4585e5f7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 10 Apr 2014 10:39:34 -0700 Subject: [PATCH 044/641] [SPARK-1276] Add a HistoryServer to render persisted UI The new feature of event logging, introduced in #42, allows the user to persist the details of his/her Spark application to storage, and later replay these events to reconstruct an after-the-fact SparkUI. Currently, however, a persisted UI can only be rendered through the standalone Master. This greatly limits the use case of this new feature as many people also run Spark on Yarn / Mesos. This PR introduces a new entity called the HistoryServer, which, given a log directory, keeps track of all completed applications independently of a Spark Master. Unlike Master, the HistoryServer needs not be running while the application is still running. It is relatively light-weight in that it only maintains static information of applications and performs no scheduling. To quickly test it out, generate event logs with ```spark.eventLog.enabled=true``` and run ```sbin/start-history-server.sh ```. Your HistoryServer awaits on port 18080. Comments and feedback are most welcome. --- A few other changes introduced in this PR include refactoring the WebUI interface, which is beginning to have a lot of duplicate code now that we have added more functionality to it. Two new SparkListenerEvents have been introduced (SparkListenerApplicationStart/End) to keep track of application name and start/finish times. This PR also clarifies the semantics of the ReplayListenerBus introduced in #42. A potential TODO in the future (not part of this PR) is to render live applications in addition to just completed applications. This is useful when applications fail, a condition that our current HistoryServer does not handle unless the user manually signals application completion (by creating the APPLICATION_COMPLETION file). Handling live applications becomes significantly more challenging, however, because it is now necessary to render the same SparkUI multiple times. To avoid reading the entire log every time, which is inefficient, we must handle reading the log from where we previously left off, but this becomes fairly complicated because we must deal with the arbitrary behavior of each input stream. Author: Andrew Or Closes #204 from andrewor14/master and squashes the following commits: 7b7234c [Andrew Or] Finished -> Completed b158d98 [Andrew Or] Address Patrick's comments 69d1b41 [Andrew Or] Do not block on posting SparkListenerApplicationEnd 19d5dd0 [Andrew Or] Merge github.com:apache/spark f7f5bf0 [Andrew Or] Make history server's web UI port a Spark configuration 2dfb494 [Andrew Or] Decouple checking for application completion from replaying d02dbaa [Andrew Or] Expose Spark version and include it in event logs 2282300 [Andrew Or] Add documentation for the HistoryServer 567474a [Andrew Or] Merge github.com:apache/spark 6edf052 [Andrew Or] Merge github.com:apache/spark 19e1fb4 [Andrew Or] Address Thomas' comments 248cb3d [Andrew Or] Limit number of live applications + add configurability a3598de [Andrew Or] Do not close file system with ReplayBus + fix bind address bc46fc8 [Andrew Or] Merge github.com:apache/spark e2f4ff9 [Andrew Or] Merge github.com:apache/spark 050419e [Andrew Or] Merge github.com:apache/spark 81b568b [Andrew Or] Fix strange error messages... 0670743 [Andrew Or] Decouple page rendering from loading files from disk 1b2f391 [Andrew Or] Minor changes a9eae7e [Andrew Or] Merge branch 'master' of github.com:apache/spark d5154da [Andrew Or] Styling and comments 5dbfbb4 [Andrew Or] Merge branch 'master' of github.com:apache/spark 60bc6d5 [Andrew Or] First complete implementation of HistoryServer (only for finished apps) 7584418 [Andrew Or] Report application start/end times to HistoryServer 8aac163 [Andrew Or] Add basic application table c086bd5 [Andrew Or] Add HistoryServer and scripts ++ Refactor WebUI interface --- bin/spark-class | 8 +- bin/spark-class2.cmd | 7 +- .../scala/org/apache/spark/SparkContext.scala | 26 +- .../spark/deploy/ApplicationDescription.scala | 4 +- .../spark/deploy/SparkUIContainer.scala | 50 +++ .../spark/deploy/history/HistoryServer.scala | 287 ++++++++++++++++++ .../history/HistoryServerArguments.scala | 76 +++++ .../spark/deploy/history/IndexPage.scala | 82 +++++ .../apache/spark/deploy/master/Master.scala | 62 ++-- .../spark/deploy/master/ui/MasterWebUI.scala | 43 +-- .../spark/deploy/worker/ui/WorkerWebUI.scala | 22 +- .../scheduler/ApplicationEventListener.scala | 50 +++ .../scheduler/EventLoggingListener.scala | 146 ++++++++- .../spark/scheduler/ReplayListenerBus.scala | 65 ++-- .../spark/scheduler/SparkListener.scala | 15 + .../spark/scheduler/SparkListenerBus.scala | 4 + .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../apache/spark/storage/FileSegment.scala | 2 +- .../scala/org/apache/spark/ui/SparkUI.scala | 49 +-- .../scala/org/apache/spark/ui/WebUI.scala | 21 +- .../apache/spark/ui/env/EnvironmentUI.scala | 3 +- .../apache/spark/ui/exec/ExecutorsUI.scala | 3 +- .../org/apache/spark/ui/jobs/IndexPage.scala | 3 +- .../apache/spark/ui/jobs/JobProgressUI.scala | 3 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 3 +- .../org/apache/spark/ui/jobs/StagePage.scala | 3 +- .../spark/ui/storage/BlockManagerUI.scala | 3 +- .../apache/spark/ui/storage/IndexPage.scala | 3 +- .../org/apache/spark/ui/storage/RDDPage.scala | 3 +- .../org/apache/spark/util/FileLogger.scala | 27 +- .../org/apache/spark/util/JsonProtocol.scala | 31 ++ .../scala/org/apache/spark/util/Utils.scala | 8 +- .../ui/jobs/JobProgressListenerSuite.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 24 +- docs/monitoring.md | 70 ++++- .../apache/spark/repl/SparkILoopInit.scala | 4 +- sbin/start-history-server.sh | 37 +++ sbin/stop-history-server.sh | 25 ++ 38 files changed, 1075 insertions(+), 201 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala create mode 100755 sbin/start-history-server.sh create mode 100755 sbin/stop-history-server.sh diff --git a/bin/spark-class b/bin/spark-class index 76fde3e448891..1b0d309cc5b1c 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -47,9 +47,9 @@ DEFAULT_MEM=${SPARK_MEM:-512m} SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" -# Add java opts and memory settings for master, worker, executors, and repl. +# Add java opts and memory settings for master, worker, history server, executors, and repl. case "$1" in - # Master and Worker use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. + # Master, Worker, and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. 'org.apache.spark.deploy.master.Master') OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_MASTER_OPTS" OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} @@ -58,6 +58,10 @@ case "$1" in OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_WORKER_OPTS" OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} ;; + 'org.apache.spark.deploy.history.HistoryServer') + OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_HISTORY_OPTS" + OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} + ;; # Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. 'org.apache.spark.executor.CoarseGrainedExecutorBackend') diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index f488cfdbeceb6..4302c1b6b7ff4 100755 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -45,14 +45,17 @@ if "x%OUR_JAVA_MEM%"=="x" set OUR_JAVA_MEM=512m set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true -rem Add java opts and memory settings for master, worker, executors, and repl. -rem Master and Worker use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. +rem Add java opts and memory settings for master, worker, history server, executors, and repl. +rem Master, Worker and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. if "%1"=="org.apache.spark.deploy.master.Master" ( set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_MASTER_OPTS% if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% ) else if "%1"=="org.apache.spark.deploy.worker.Worker" ( set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_WORKER_OPTS% if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% +) else if "%1"=="org.apache.spark.deploy.history.HistoryServer" ( + set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_HISTORY_OPTS% + if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% rem Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. ) else if "%1"=="org.apache.spark.executor.CoarseGrainedExecutorBackend" ( diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 76305237b03d5..e6c9b7000d819 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -219,15 +219,12 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] val eventLogger: Option[EventLoggingListener] = { if (conf.getBoolean("spark.eventLog.enabled", false)) { val logger = new EventLoggingListener(appName, conf) + logger.start() listenerBus.addListener(logger) Some(logger) } else None } - // Information needed to replay logged events, if any - private[spark] val eventLoggingInfo: Option[EventLoggingInfo] = - eventLogger.map { logger => Some(logger.info) }.getOrElse(None) - // At this point, all relevant SparkListeners have been registered, so begin releasing events listenerBus.start() @@ -292,6 +289,7 @@ class SparkContext(config: SparkConf) extends Logging { cleaner.foreach(_.start()) postEnvironmentUpdate() + postApplicationStart() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration: Configuration = { @@ -777,6 +775,9 @@ class SparkContext(config: SparkConf) extends Logging { listenerBus.addListener(listener) } + /** The version of Spark on which this application is running. */ + def version = SparkContext.SPARK_VERSION + /** * Return a map from the slave to the max memory available for caching and the remaining * memory available for caching. @@ -930,6 +931,7 @@ class SparkContext(config: SparkConf) extends Logging { /** Shut down the SparkContext. */ def stop() { + postApplicationEnd() ui.stop() // Do this only if not stopped already - best case effort. // prevent NPE if stopped more than once. @@ -1175,6 +1177,20 @@ class SparkContext(config: SparkConf) extends Logging { /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() + /** Post the application start event */ + private def postApplicationStart() { + listenerBus.post(SparkListenerApplicationStart(appName, startTime, sparkUser)) + } + + /** + * Post the application end event to all listeners immediately, rather than adding it + * to the event queue for it to be asynchronously processed eventually. Otherwise, a race + * condition exists in which the listeners may stop before this event has been propagated. + */ + private def postApplicationEnd() { + listenerBus.post(SparkListenerApplicationEnd(System.currentTimeMillis)) + } + /** Post the environment update event once the task scheduler is ready */ private def postEnvironmentUpdate() { if (taskScheduler != null) { @@ -1200,6 +1216,8 @@ class SparkContext(config: SparkConf) extends Logging { */ object SparkContext extends Logging { + private[spark] val SPARK_VERSION = "1.0.0" + private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description" private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id" diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 15fa8a7679874..86305d2ea8a09 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy -import org.apache.spark.scheduler.EventLoggingInfo - private[spark] class ApplicationDescription( val name: String, val maxCores: Option[Int], @@ -26,7 +24,7 @@ private[spark] class ApplicationDescription( val command: Command, val sparkHome: Option[String], var appUiUrl: String, - val eventLogInfo: Option[EventLoggingInfo] = None) + val eventLogDir: Option[String] = None) extends Serializable { val user = System.getProperty("user.name", "") diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala b/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala new file mode 100644 index 0000000000000..33fceae4ff489 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala @@ -0,0 +1,50 @@ +/* + * 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.deploy + +import org.apache.spark.ui.{SparkUI, WebUI} + +private[spark] abstract class SparkUIContainer(name: String) extends WebUI(name) { + + /** Attach a SparkUI to this container. Only valid after bind(). */ + def attachUI(ui: SparkUI) { + assert(serverInfo.isDefined, + "%s must be bound to a server before attaching SparkUIs".format(name)) + val rootHandler = serverInfo.get.rootHandler + for (handler <- ui.handlers) { + rootHandler.addHandler(handler) + if (!handler.isStarted) { + handler.start() + } + } + } + + /** Detach a SparkUI from this container. Only valid after bind(). */ + def detachUI(ui: SparkUI) { + assert(serverInfo.isDefined, + "%s must be bound to a server before detaching SparkUIs".format(name)) + val rootHandler = serverInfo.get.rootHandler + for (handler <- ui.handlers) { + if (handler.isStarted) { + handler.stop() + } + rootHandler.removeHandler(handler) + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala new file mode 100644 index 0000000000000..97d2ba9deed33 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -0,0 +1,287 @@ +/* + * 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.deploy.history + +import javax.servlet.http.HttpServletRequest + +import scala.collection.mutable + +import org.apache.hadoop.fs.{FileStatus, Path} +import org.eclipse.jetty.servlet.ServletContextHandler + +import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.deploy.SparkUIContainer +import org.apache.spark.scheduler._ +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.Utils + +/** + * A web server that renders SparkUIs of completed applications. + * + * For the standalone mode, MasterWebUI already achieves this functionality. Thus, the + * main use case of the HistoryServer is in other deploy modes (e.g. Yarn or Mesos). + * + * The logging directory structure is as follows: Within the given base directory, each + * application's event logs are maintained in the application's own sub-directory. This + * is the same structure as maintained in the event log write code path in + * EventLoggingListener. + * + * @param baseLogDir The base directory in which event logs are found + */ +class HistoryServer( + val baseLogDir: String, + conf: SparkConf) + extends SparkUIContainer("History Server") with Logging { + + import HistoryServer._ + + private val fileSystem = Utils.getHadoopFileSystem(baseLogDir) + private val localHost = Utils.localHostName() + private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) + private val port = WEB_UI_PORT + private val securityManager = new SecurityManager(conf) + private val indexPage = new IndexPage(this) + + // A timestamp of when the disk was last accessed to check for log updates + private var lastLogCheckTime = -1L + + // Number of completed applications found in this directory + private var numCompletedApplications = 0 + + @volatile private var stopped = false + + /** + * A background thread that periodically checks for event log updates on disk. + * + * If a log check is invoked manually in the middle of a period, this thread re-adjusts the + * time at which it performs the next log check to maintain the same period as before. + * + * TODO: Add a mechanism to update manually. + */ + private val logCheckingThread = new Thread { + override def run() { + while (!stopped) { + val now = System.currentTimeMillis + if (now - lastLogCheckTime > UPDATE_INTERVAL_MS) { + checkForLogs() + Thread.sleep(UPDATE_INTERVAL_MS) + } else { + // If the user has manually checked for logs recently, wait until + // UPDATE_INTERVAL_MS after the last check time + Thread.sleep(lastLogCheckTime + UPDATE_INTERVAL_MS - now) + } + } + } + } + + private val handlers = Seq[ServletContextHandler]( + createStaticHandler(STATIC_RESOURCE_DIR, "/static"), + createServletHandler("/", + (request: HttpServletRequest) => indexPage.render(request), securityMgr = securityManager) + ) + + // A mapping of application ID to its history information, which includes the rendered UI + val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]() + + /** + * Start the history server. + * + * This starts a background thread that periodically synchronizes information displayed on + * this UI with the event logs in the provided base directory. + */ + def start() { + logCheckingThread.start() + } + + /** Bind to the HTTP server behind this web interface. */ + override def bind() { + try { + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf)) + logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort)) + } catch { + case e: Exception => + logError("Failed to bind HistoryServer", e) + System.exit(1) + } + } + + /** + * Check for any updates to event logs in the base directory. This is only effective once + * the server has been bound. + * + * If a new completed application is found, the server renders the associated SparkUI + * from the application's event logs, attaches this UI to itself, and stores metadata + * information for this application. + * + * If the logs for an existing completed application are no longer found, the server + * removes all associated information and detaches the SparkUI. + */ + def checkForLogs() = synchronized { + if (serverInfo.isDefined) { + lastLogCheckTime = System.currentTimeMillis + logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTime)) + try { + val logStatus = fileSystem.listStatus(new Path(baseLogDir)) + val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() + val logInfos = logDirs + .sortBy { dir => getModificationTime(dir) } + .map { dir => (dir, EventLoggingListener.parseLoggingInfo(dir.getPath, fileSystem)) } + .filter { case (dir, info) => info.applicationComplete } + + // Logging information for applications that should be retained + val retainedLogInfos = logInfos.takeRight(RETAINED_APPLICATIONS) + val retainedAppIds = retainedLogInfos.map { case (dir, _) => dir.getPath.getName } + + // Remove any applications that should no longer be retained + appIdToInfo.foreach { case (appId, info) => + if (!retainedAppIds.contains(appId)) { + detachUI(info.ui) + appIdToInfo.remove(appId) + } + } + + // Render the application's UI if it is not already there + retainedLogInfos.foreach { case (dir, info) => + val appId = dir.getPath.getName + if (!appIdToInfo.contains(appId)) { + renderSparkUI(dir, info) + } + } + + // Track the total number of completed applications observed this round + numCompletedApplications = logInfos.size + + } catch { + case t: Throwable => logError("Exception in checking for event log updates", t) + } + } else { + logWarning("Attempted to check for event log updates before binding the server.") + } + } + + /** + * Render a new SparkUI from the event logs if the associated application is completed. + * + * HistoryServer looks for a special file that indicates application completion in the given + * directory. If this file exists, the associated application is regarded to be completed, in + * which case the server proceeds to render the SparkUI. Otherwise, the server does nothing. + */ + private def renderSparkUI(logDir: FileStatus, logInfo: EventLoggingInfo) { + val path = logDir.getPath + val appId = path.getName + val replayBus = new ReplayListenerBus(logInfo.logPaths, fileSystem, logInfo.compressionCodec) + val ui = new SparkUI(replayBus, appId, "/history/" + appId) + val appListener = new ApplicationEventListener + replayBus.addListener(appListener) + + // Do not call ui.bind() to avoid creating a new server for each application + ui.start() + replayBus.replay() + if (appListener.applicationStarted) { + attachUI(ui) + val appName = appListener.appName + val sparkUser = appListener.sparkUser + val startTime = appListener.startTime + val endTime = appListener.endTime + val lastUpdated = getModificationTime(logDir) + ui.setAppName(appName + " (completed)") + appIdToInfo(appId) = ApplicationHistoryInfo(appId, appName, startTime, endTime, + lastUpdated, sparkUser, path, ui) + } + } + + /** Stop the server and close the file system. */ + override def stop() { + super.stop() + stopped = true + fileSystem.close() + } + + /** Return the address of this server. */ + def getAddress: String = "http://" + publicHost + ":" + boundPort + + /** Return the number of completed applications found, whether or not the UI is rendered. */ + def getNumApplications: Int = numCompletedApplications + + /** Return when this directory was last modified. */ + private def getModificationTime(dir: FileStatus): Long = { + try { + val logFiles = fileSystem.listStatus(dir.getPath) + if (logFiles != null && !logFiles.isEmpty) { + logFiles.map(_.getModificationTime).max + } else { + dir.getModificationTime + } + } catch { + case t: Throwable => + logError("Exception in accessing modification time of %s".format(dir.getPath), t) + -1L + } + } +} + +/** + * The recommended way of starting and stopping a HistoryServer is through the scripts + * start-history-server.sh and stop-history-server.sh. The path to a base log directory + * is must be specified, while the requested UI port is optional. For example: + * + * ./sbin/spark-history-server.sh /tmp/spark-events + * ./sbin/spark-history-server.sh hdfs://1.2.3.4:9000/spark-events + * + * This launches the HistoryServer as a Spark daemon. + */ +object HistoryServer { + private val conf = new SparkConf + + // Interval between each check for event log updates + val UPDATE_INTERVAL_MS = conf.getInt("spark.history.updateInterval", 10) * 1000 + + // How many applications to retain + val RETAINED_APPLICATIONS = conf.getInt("spark.history.retainedApplications", 250) + + // The port to which the web UI is bound + val WEB_UI_PORT = conf.getInt("spark.history.ui.port", 18080) + + val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR + + def main(argStrings: Array[String]) { + val args = new HistoryServerArguments(argStrings) + val server = new HistoryServer(args.logDir, conf) + server.bind() + server.start() + + // Wait until the end of the world... or if the HistoryServer process is manually stopped + while(true) { Thread.sleep(Int.MaxValue) } + server.stop() + } +} + + +private[spark] case class ApplicationHistoryInfo( + id: String, + name: String, + startTime: Long, + endTime: Long, + lastUpdated: Long, + sparkUser: String, + logDirPath: Path, + ui: SparkUI) { + def started = startTime != -1 + def completed = endTime != -1 +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala new file mode 100644 index 0000000000000..943c061743dbd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -0,0 +1,76 @@ +/* + * 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.deploy.history + +import java.net.URI + +import org.apache.hadoop.fs.Path + +import org.apache.spark.util.Utils + +/** + * Command-line parser for the master. + */ +private[spark] class HistoryServerArguments(args: Array[String]) { + var logDir = "" + + parse(args.toList) + + private def parse(args: List[String]): Unit = { + args match { + case ("--dir" | "-d") :: value :: tail => + logDir = value + parse(tail) + + case ("--help" | "-h") :: tail => + printUsageAndExit(0) + + case Nil => + + case _ => + printUsageAndExit(1) + } + validateLogDir() + } + + private def validateLogDir() { + if (logDir == "") { + System.err.println("Logging directory must be specified.") + printUsageAndExit(1) + } + val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) + val path = new Path(logDir) + if (!fileSystem.exists(path)) { + System.err.println("Logging directory specified does not exist: %s".format(logDir)) + printUsageAndExit(1) + } + if (!fileSystem.getFileStatus(path).isDir) { + System.err.println("Logging directory specified is not a directory: %s".format(logDir)) + printUsageAndExit(1) + } + } + + private def printUsageAndExit(exitCode: Int) { + System.err.println( + "Usage: HistoryServer [options]\n" + + "\n" + + "Options:\n" + + " -d DIR, --dir DIR Location of event log files") + System.exit(exitCode) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala new file mode 100644 index 0000000000000..54dffffec71c5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala @@ -0,0 +1,82 @@ +/* + * 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.deploy.history + +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.ui.{UIUtils, WebUI} + +private[spark] class IndexPage(parent: HistoryServer) { + + def render(request: HttpServletRequest): Seq[Node] = { + val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated } + val appTable = UIUtils.listingTable(appHeader, appRow, appRows) + val content = +
+
+
    +
  • Event Log Location: {parent.baseLogDir}
  • +
+ { + if (parent.appIdToInfo.size > 0) { +

+ Showing {parent.appIdToInfo.size}/{parent.getNumApplications} + Completed Application{if (parent.getNumApplications > 1) "s" else ""} +

++ + appTable + } else { +

No Completed Applications Found

+ } + } +
+
+ UIUtils.basicSparkPage(content, "History Server") + } + + private val appHeader = Seq( + "App Name", + "Started", + "Completed", + "Duration", + "Spark User", + "Log Directory", + "Last Updated") + + private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { + val appName = if (info.started) info.name else info.logDirPath.getName + val uiAddress = parent.getAddress + info.ui.basePath + val startTime = if (info.started) WebUI.formatDate(info.startTime) else "Not started" + val endTime = if (info.completed) WebUI.formatDate(info.endTime) else "Not completed" + val difference = if (info.started && info.completed) info.endTime - info.startTime else -1L + val duration = if (difference > 0) WebUI.formatDuration(difference) else "---" + val sparkUser = if (info.started) info.sparkUser else "Unknown user" + val logDirectory = info.logDirPath.getName + val lastUpdated = WebUI.formatDate(info.lastUpdated) + + {appName} + {startTime} + {endTime} + {duration} + {sparkUser} + {logDirectory} + {lastUpdated} + + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 95bd62e88db2b..2446e86cb6672 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -29,6 +29,7 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension +import org.apache.hadoop.fs.FileSystem import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState} @@ -37,7 +38,7 @@ import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.scheduler.ReplayListenerBus +import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{AkkaUtils, Utils} @@ -45,7 +46,8 @@ private[spark] class Master( host: String, port: Int, webUiPort: Int, - val securityMgr: SecurityManager) extends Actor with Logging { + val securityMgr: SecurityManager) + extends Actor with Logging { import context.dispatcher // to use Akka's scheduler.schedule() @@ -71,6 +73,7 @@ private[spark] class Master( var nextAppNumber = 0 val appIdToUI = new HashMap[String, SparkUI] + val fileSystemsUsed = new HashSet[FileSystem] val drivers = new HashSet[DriverInfo] val completedDrivers = new ArrayBuffer[DriverInfo] @@ -149,6 +152,7 @@ private[spark] class Master( override def postStop() { webUi.stop() + fileSystemsUsed.foreach(_.close()) masterMetricsSystem.stop() applicationMetricsSystem.stop() persistenceEngine.close() @@ -630,11 +634,7 @@ private[spark] class Master( waitingApps -= app // If application events are logged, use them to rebuild the UI - startPersistedSparkUI(app).map { ui => - app.desc.appUiUrl = ui.basePath - appIdToUI(app.id) = ui - webUi.attachUI(ui) - }.getOrElse { + if (!rebuildSparkUI(app)) { // Avoid broken links if the UI is not reconstructed app.desc.appUiUrl = "" } @@ -654,30 +654,34 @@ private[spark] class Master( } /** - * Start a new SparkUI rendered from persisted storage. If this is unsuccessful for any reason, - * return None. Otherwise return the reconstructed UI. + * Rebuild a new SparkUI from the given application's event logs. + * Return whether this is successful. */ - def startPersistedSparkUI(app: ApplicationInfo): Option[SparkUI] = { + def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name - val eventLogInfo = app.desc.eventLogInfo.getOrElse { return None } - val eventLogDir = eventLogInfo.logDir - val eventCompressionCodec = eventLogInfo.compressionCodec - val appConf = new SparkConf - eventCompressionCodec.foreach { codec => - appConf.set("spark.eventLog.compress", "true") - appConf.set("spark.io.compression.codec", codec) - } - val replayerBus = new ReplayListenerBus(appConf) - val ui = new SparkUI( - appConf, - replayerBus, - "%s (finished)".format(appName), - "/history/%s".format(app.id)) - - // Do not call ui.bind() to avoid creating a new server for each application - ui.start() - val success = replayerBus.replay(eventLogDir) - if (success) Some(ui) else None + val eventLogDir = app.desc.eventLogDir.getOrElse { return false } + val fileSystem = Utils.getHadoopFileSystem(eventLogDir) + val eventLogInfo = EventLoggingListener.parseLoggingInfo(eventLogDir, fileSystem) + val eventLogPaths = eventLogInfo.logPaths + val compressionCodec = eventLogInfo.compressionCodec + if (!eventLogPaths.isEmpty) { + try { + val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) + val ui = new SparkUI(replayBus, appName + " (completed)", "/history/" + app.id) + ui.start() + replayBus.replay() + app.desc.appUiUrl = ui.basePath + appIdToUI(app.id) = ui + webUi.attachUI(ui) + return true + } catch { + case t: Throwable => + logError("Exception in replaying log for application %s (%s)".format(appName, app.id), t) + } + } else { + logWarning("Application %s (%s) has no valid logs: %s".format(appName, app.id, eventLogDir)) + } + false } /** Generate a new app ID given a app's submission date */ diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 01d9f52f4b7b4..30c8ade408a5a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -22,8 +22,9 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.Logging +import org.apache.spark.deploy.SparkUIContainer import org.apache.spark.deploy.master.Master -import org.apache.spark.ui.{ServerInfo, SparkUI} +import org.apache.spark.ui.SparkUI import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -31,7 +32,9 @@ import org.apache.spark.util.{AkkaUtils, Utils} * Web UI server for the standalone master. */ private[spark] -class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { +class MasterWebUI(val master: Master, requestedPort: Int) + extends SparkUIContainer("MasterWebUI") with Logging { + val masterActorRef = master.self val timeout = AkkaUtils.askTimeout(master.conf) @@ -39,7 +42,6 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { private val port = requestedPort private val applicationPage = new ApplicationPage(this) private val indexPage = new IndexPage(this) - private var serverInfo: Option[ServerInfo] = None private val handlers: Seq[ServletContextHandler] = { master.masterMetricsSystem.getServletHandlers ++ @@ -57,47 +59,18 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { ) } - def bind() { + /** Bind to the HTTP server behind this web interface. */ + override def bind() { try { serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, master.conf)) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Master JettyUtils", e) + logError("Failed to create Master web UI", e) System.exit(1) } } - def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - - /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ - def attachUI(ui: SparkUI) { - assert(serverInfo.isDefined, "Master UI must be bound to a server before attaching SparkUIs") - val rootHandler = serverInfo.get.rootHandler - for (handler <- ui.handlers) { - rootHandler.addHandler(handler) - if (!handler.isStarted) { - handler.start() - } - } - } - - /** Detach a reconstructed UI from this Master UI. Only valid after bind(). */ - def detachUI(ui: SparkUI) { - assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") - val rootHandler = serverInfo.get.rootHandler - for (handler <- ui.handlers) { - if (handler.isStarted) { - handler.stop() - } - rootHandler.removeHandler(handler) - } - } - - def stop() { - assert(serverInfo.isDefined, "Attempted to stop a Master UI that was not bound to a server!") - serverInfo.get.server.stop() - } } private[spark] object MasterWebUI { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 650f3da5ce3ff..5625a44549aaa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -24,7 +24,7 @@ import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.Logging import org.apache.spark.deploy.worker.Worker -import org.apache.spark.ui.{JettyUtils, ServerInfo, SparkUI, UIUtils} +import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -33,15 +33,14 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) - extends Logging { + extends WebUI("WorkerWebUI") with Logging { val timeout = AkkaUtils.askTimeout(worker.conf) private val host = Utils.localHostName() private val port = requestedPort.getOrElse( - worker.conf.get("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) + worker.conf.getInt("worker.ui.port", WorkerWebUI.DEFAULT_PORT)) private val indexPage = new IndexPage(this) - private var serverInfo: Option[ServerInfo] = None private val handlers: Seq[ServletContextHandler] = { worker.metricsSystem.getServletHandlers ++ @@ -58,19 +57,18 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I ) } - def bind() { + /** Bind to the HTTP server behind this web interface. */ + override def bind() { try { - serverInfo = Some(JettyUtils.startJettyServer("0.0.0.0", port, handlers, worker.conf)) + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, worker.conf)) logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Worker JettyUtils", e) + logError("Failed to create Worker web UI", e) System.exit(1) } } - def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - private def log(request: HttpServletRequest): String = { val defaultBytes = 100 * 1024 @@ -187,13 +185,9 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I (startByte, endByte) } - def stop() { - assert(serverInfo.isDefined, "Attempted to stop a Worker UI that was not bound to a server!") - serverInfo.get.server.stop() - } } private[spark] object WorkerWebUI { + val DEFAULT_PORT=8081 val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR - val DEFAULT_PORT="8081" } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala new file mode 100644 index 0000000000000..affda13df6531 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -0,0 +1,50 @@ +/* + * 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.scheduler + +/** + * A simple listener for application events. + * + * This listener expects to hear events from a single application only. If events + * from multiple applications are seen, the behavior is unspecified. + */ +private[spark] class ApplicationEventListener extends SparkListener { + var appName = "" + var sparkUser = "" + var startTime = -1L + var endTime = -1L + + def applicationStarted = startTime != -1 + + def applicationFinished = endTime != -1 + + def applicationDuration: Long = { + val difference = endTime - startTime + if (applicationStarted && applicationFinished && difference > 0) difference else -1L + } + + override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { + appName = applicationStart.appName + startTime = applicationStart.time + sparkUser = applicationStart.sparkUser + } + + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { + endTime = applicationEnd.time + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 217f8825c2ae9..b983c16af14f4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -17,11 +17,14 @@ package org.apache.spark.scheduler +import scala.collection.mutable + +import org.apache.hadoop.fs.{FileSystem, Path} import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{JsonProtocol, FileLogger} +import org.apache.spark.util.{FileLogger, JsonProtocol} /** * A SparkListener that logs events to persistent storage. @@ -36,6 +39,8 @@ import org.apache.spark.util.{JsonProtocol, FileLogger} private[spark] class EventLoggingListener(appName: String, conf: SparkConf) extends SparkListener with Logging { + import EventLoggingListener._ + private val shouldCompress = conf.getBoolean("spark.eventLog.compress", false) private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", false) private val outputBufferSize = conf.getInt("spark.eventLog.buffer.kb", 100) * 1024 @@ -46,17 +51,21 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) private val logger = new FileLogger(logDir, conf, outputBufferSize, shouldCompress, shouldOverwrite) - // Information needed to replay the events logged by this listener later - val info = { - val compressionCodec = if (shouldCompress) { - Some(conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC)) - } else None - EventLoggingInfo(logDir, compressionCodec) + /** + * Begin logging events. + * If compression is used, log a file that indicates which compression library is used. + */ + def start() { + logInfo("Logging events to %s".format(logDir)) + if (shouldCompress) { + val codec = conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC) + logger.newFile(COMPRESSION_CODEC_PREFIX + codec) + } + logger.newFile(SPARK_VERSION_PREFIX + SparkContext.SPARK_VERSION) + logger.newFile(LOG_PREFIX + logger.fileIndex) } - logInfo("Logging events to %s".format(logDir)) - - /** Log the event as JSON */ + /** Log the event as JSON. */ private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { val eventJson = compact(render(JsonProtocol.sparkEventToJson(event))) logger.logLine(eventJson) @@ -90,9 +99,118 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) logEvent(event, flushLogger = true) override def onUnpersistRDD(event: SparkListenerUnpersistRDD) = logEvent(event, flushLogger = true) + override def onApplicationStart(event: SparkListenerApplicationStart) = + logEvent(event, flushLogger = true) + override def onApplicationEnd(event: SparkListenerApplicationEnd) = + logEvent(event, flushLogger = true) + + /** + * Stop logging events. + * In addition, create an empty special file to indicate application completion. + */ + def stop() = { + logger.newFile(APPLICATION_COMPLETE) + logger.stop() + } +} + +private[spark] object EventLoggingListener extends Logging { + val LOG_PREFIX = "EVENT_LOG_" + val SPARK_VERSION_PREFIX = "SPARK_VERSION_" + val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" + val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" + + // A cache for compression codecs to avoid creating the same codec many times + private val codecMap = new mutable.HashMap[String, CompressionCodec] + + def isEventLogFile(fileName: String): Boolean = { + fileName.startsWith(LOG_PREFIX) + } + + def isSparkVersionFile(fileName: String): Boolean = { + fileName.startsWith(SPARK_VERSION_PREFIX) + } + + def isCompressionCodecFile(fileName: String): Boolean = { + fileName.startsWith(COMPRESSION_CODEC_PREFIX) + } + + def isApplicationCompleteFile(fileName: String): Boolean = { + fileName == APPLICATION_COMPLETE + } + + def parseSparkVersion(fileName: String): String = { + if (isSparkVersionFile(fileName)) { + fileName.replaceAll(SPARK_VERSION_PREFIX, "") + } else "" + } + + def parseCompressionCodec(fileName: String): String = { + if (isCompressionCodecFile(fileName)) { + fileName.replaceAll(COMPRESSION_CODEC_PREFIX, "") + } else "" + } + + /** + * Parse the event logging information associated with the logs in the given directory. + * + * Specifically, this looks for event log files, the Spark version file, the compression + * codec file (if event logs are compressed), and the application completion file (if the + * application has run to completion). + */ + def parseLoggingInfo(logDir: Path, fileSystem: FileSystem): EventLoggingInfo = { + try { + val fileStatuses = fileSystem.listStatus(logDir) + val filePaths = + if (fileStatuses != null) { + fileStatuses.filter(!_.isDir).map(_.getPath).toSeq + } else { + Seq[Path]() + } + if (filePaths.isEmpty) { + logWarning("No files found in logging directory %s".format(logDir)) + } + EventLoggingInfo( + logPaths = filePaths.filter { path => isEventLogFile(path.getName) }, + sparkVersion = filePaths + .find { path => isSparkVersionFile(path.getName) } + .map { path => parseSparkVersion(path.getName) } + .getOrElse(""), + compressionCodec = filePaths + .find { path => isCompressionCodecFile(path.getName) } + .map { path => + val codec = EventLoggingListener.parseCompressionCodec(path.getName) + val conf = new SparkConf + conf.set("spark.io.compression.codec", codec) + codecMap.getOrElseUpdate(codec, CompressionCodec.createCodec(conf)) + }, + applicationComplete = filePaths.exists { path => isApplicationCompleteFile(path.getName) } + ) + } catch { + case t: Throwable => + logError("Exception in parsing logging info from directory %s".format(logDir), t) + EventLoggingInfo.empty + } + } - def stop() = logger.stop() + /** + * Parse the event logging information associated with the logs in the given directory. + */ + def parseLoggingInfo(logDir: String, fileSystem: FileSystem): EventLoggingInfo = { + parseLoggingInfo(new Path(logDir), fileSystem) + } } -// If compression is not enabled, compressionCodec is None -private[spark] case class EventLoggingInfo(logDir: String, compressionCodec: Option[String]) + +/** + * Information needed to process the event logs associated with an application. + */ +private[spark] case class EventLoggingInfo( + logPaths: Seq[Path], + sparkVersion: String, + compressionCodec: Option[CompressionCodec], + applicationComplete: Boolean = false) + +private[spark] object EventLoggingInfo { + def empty = EventLoggingInfo(Seq[Path](), "", None, applicationComplete = false) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index db76178b65501..b03665fd56d33 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler import java.io.InputStream -import java.net.URI import scala.io.Source @@ -26,63 +25,47 @@ import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.hadoop.fs.{Path, FileSystem} import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.Logging import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{JsonProtocol, Utils} +import org.apache.spark.util.JsonProtocol /** - * An EventBus that replays logged events from persisted storage + * A SparkListenerBus that replays logged events from persisted storage. + * + * This class expects files to be appropriately prefixed as specified in EventLoggingListener. + * There exists a one-to-one mapping between ReplayListenerBus and event logging applications. */ -private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus with Logging { - private val compressed = conf.getBoolean("spark.eventLog.compress", false) +private[spark] class ReplayListenerBus( + logPaths: Seq[Path], + fileSystem: FileSystem, + compressionCodec: Option[CompressionCodec]) + extends SparkListenerBus with Logging { - // Only used if compression is enabled - private lazy val compressionCodec = CompressionCodec.createCodec(conf) + private var replayed = false - /** - * Return a list of paths representing log files in the given directory. - */ - private def getLogFilePaths(logDir: String, fileSystem: FileSystem): Array[Path] = { - val path = new Path(logDir) - if (!fileSystem.exists(path) || !fileSystem.getFileStatus(path).isDir) { - logWarning("Log path provided is not a valid directory: %s".format(logDir)) - return Array[Path]() - } - val logStatus = fileSystem.listStatus(path) - if (logStatus == null || !logStatus.exists(!_.isDir)) { - logWarning("Log path provided contains no log files: %s".format(logDir)) - return Array[Path]() - } - logStatus.filter(!_.isDir).map(_.getPath).sortBy(_.getName) + if (logPaths.length == 0) { + logWarning("Log path provided contains no log files.") } /** * Replay each event in the order maintained in the given logs. + * This should only be called exactly once. */ - def replay(logDir: String): Boolean = { - val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) - val logPaths = getLogFilePaths(logDir, fileSystem) - if (logPaths.length == 0) { - return false - } - + def replay() { + assert(!replayed, "ReplayListenerBus cannot replay events more than once") logPaths.foreach { path => // Keep track of input streams at all levels to close them later // This is necessary because an exception can occur in between stream initializations var fileStream: Option[InputStream] = None var bufferedStream: Option[InputStream] = None var compressStream: Option[InputStream] = None - var currentLine = "" + var currentLine = "" try { - currentLine = "" fileStream = Some(fileSystem.open(path)) bufferedStream = Some(new FastBufferedInputStream(fileStream.get)) - compressStream = - if (compressed) { - Some(compressionCodec.compressedInputStream(bufferedStream.get)) - } else bufferedStream + compressStream = Some(wrapForCompression(bufferedStream.get)) - // Parse each line as an event and post it to all attached listeners + // Parse each line as an event and post the event to all attached listeners val lines = Source.fromInputStream(compressStream.get).getLines() lines.foreach { line => currentLine = line @@ -98,7 +81,11 @@ private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus compressStream.foreach(_.close()) } } - fileSystem.close() - true + replayed = true + } + + /** If a compression codec is specified, wrap the given stream in a compression stream. */ + private def wrapForCompression(stream: InputStream): InputStream = { + compressionCodec.map(_.compressedInputStream(stream)).getOrElse(stream) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index ced20350d5356..378cf1aaebe7b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -75,6 +75,11 @@ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) @DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent +case class SparkListenerApplicationStart(appName: String, time: Long, sparkUser: String) + extends SparkListenerEvent + +case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent + /** An event used in the listener to shutdown the listener daemon thread. */ private[spark] case object SparkListenerShutdown extends SparkListenerEvent @@ -141,6 +146,16 @@ trait SparkListener { * Called when an RDD is manually unpersisted by the application */ def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { } + + /** + * Called when the application starts + */ + def onApplicationStart(applicationStart: SparkListenerApplicationStart) { } + + /** + * Called when the application ends + */ + def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 729e120497571..d6df193d9bcf8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -61,6 +61,10 @@ private[spark] trait SparkListenerBus { sparkListeners.foreach(_.onBlockManagerRemoved(blockManagerRemoved)) case unpersistRDD: SparkListenerUnpersistRDD => sparkListeners.foreach(_.onUnpersistRDD(unpersistRDD)) + case applicationStart: SparkListenerApplicationStart => + sparkListeners.foreach(_.onApplicationStart(applicationStart)) + case applicationEnd: SparkListenerApplicationEnd => + sparkListeners.foreach(_.onApplicationEnd(applicationEnd)) case SparkListenerShutdown => } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 25b7472a99cdb..936e9db80573d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -49,7 +49,7 @@ private[spark] class SparkDeploySchedulerBackend( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - sparkHome, sc.ui.appUIAddress, sc.eventLoggingInfo) + sparkHome, sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() diff --git a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala index 555486830a769..132502b75f8cd 100644 --- a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala +++ b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala @@ -23,6 +23,6 @@ import java.io.File * References a particular segment of a file (potentially the entire file), * based off an offset and a length. */ -private[spark] class FileSegment(val file: File, val offset: Long, val length : Long) { +private[spark] class FileSegment(val file: File, val offset: Long, val length: Long) { override def toString = "(name=%s, offset=%d, length=%d)".format(file.getName, offset, length) } diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index f53df7fbedf39..b8e6e15880bf5 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -34,23 +34,22 @@ private[spark] class SparkUI( val sc: SparkContext, conf: SparkConf, val listenerBus: SparkListenerBus, - val appName: String, + var appName: String, val basePath: String = "") - extends Logging { + extends WebUI("SparkUI") with Logging { def this(sc: SparkContext) = this(sc, sc.conf, sc.listenerBus, sc.appName) - def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = - this(null, conf, listenerBus, appName, basePath) + def this(listenerBus: SparkListenerBus, appName: String, basePath: String) = + this(null, new SparkConf, listenerBus, appName, basePath) // If SparkContext is not provided, assume the associated application is not live val live = sc != null val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) - private val bindHost = Utils.localHostName() - private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost) - private val port = conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt - private var serverInfo: Option[ServerInfo] = None + private val localHost = Utils.localHostName() + private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) + private val port = conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) private val storage = new BlockManagerUI(this) private val jobs = new JobProgressUI(this) @@ -77,20 +76,10 @@ private[spark] class SparkUI( // Maintain executor storage status through Spark events val storageStatusListener = new StorageStatusListener - /** Bind the HTTP server which backs this web interface */ - def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, sc.conf)) - logInfo("Started Spark Web UI at http://%s:%d".format(publicHost, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Spark JettyUtils", e) - System.exit(1) - } + def setAppName(name: String) { + appName = name } - def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - /** Initialize all components of the server */ def start() { storage.start() @@ -106,9 +95,21 @@ private[spark] class SparkUI( listenerBus.addListener(exec.listener) } - def stop() { - assert(serverInfo.isDefined, "Attempted to stop a SparkUI that was not bound to a server!") - serverInfo.get.server.stop() + /** Bind to the HTTP server behind this web interface. */ + override def bind() { + try { + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, sc.conf)) + logInfo("Started Spark web UI at http://%s:%d".format(publicHost, boundPort)) + } catch { + case e: Exception => + logError("Failed to create Spark web UI", e) + System.exit(1) + } + } + + /** Stop the server behind this web interface. Only valid after bind(). */ + override def stop() { + super.stop() logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) } @@ -117,6 +118,6 @@ private[spark] class SparkUI( } private[spark] object SparkUI { - val DEFAULT_PORT = "4040" + val DEFAULT_PORT = 4040 val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" } diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index a7b872f3445a4..2cc7582eca8a3 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -20,6 +20,25 @@ package org.apache.spark.ui import java.text.SimpleDateFormat import java.util.Date +private[spark] abstract class WebUI(name: String) { + protected var serverInfo: Option[ServerInfo] = None + + /** + * Bind to the HTTP server behind this web interface. + * Overridden implementation should set serverInfo. + */ + def bind() { } + + /** Return the actual port to which this server is bound. Only valid after bind(). */ + def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) + + /** Stop the server behind this web interface. Only valid after bind(). */ + def stop() { + assert(serverInfo.isDefined, "Attempted to stop %s before binding to a server!".format(name)) + serverInfo.get.server.stop() + } +} + /** * Utilities used throughout the web UI. */ @@ -45,6 +64,6 @@ private[spark] object WebUI { return "%.0f min".format(minutes) } val hours = minutes / 60 - return "%.1f h".format(hours) + "%.1f h".format(hours) } } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 23e90c34d5b33..33df97187ea78 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -29,10 +29,11 @@ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Environment private[ui] class EnvironmentUI(parent: SparkUI) { - private val appName = parent.appName private val basePath = parent.basePath private var _listener: Option[EnvironmentListener] = None + private def appName = parent.appName + lazy val listener = _listener.get def start() { diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 031ed88a493a8..77a38a1d3aa7c 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -33,10 +33,11 @@ import org.apache.spark.ui.{SparkUI, UIUtils} import org.apache.spark.util.Utils private[ui] class ExecutorsUI(parent: SparkUI) { - private val appName = parent.appName private val basePath = parent.basePath private var _listener: Option[ExecutorsListener] = None + private def appName = parent.appName + lazy val listener = _listener.get def start() { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 70d62b66a4829..f811aff616bcf 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -27,13 +27,14 @@ import org.apache.spark.ui.UIUtils /** Page showing list of all ongoing and recently finished stages and pools */ private[ui] class IndexPage(parent: JobProgressUI) { - private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc private lazy val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val activeStages = listener.activeStages.values.toSeq diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index b2c67381cc3da..ad1a12cdc4e36 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -29,7 +29,6 @@ import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ private[ui] class JobProgressUI(parent: SparkUI) { - val appName = parent.appName val basePath = parent.basePath val live = parent.live val sc = parent.sc @@ -42,6 +41,8 @@ private[ui] class JobProgressUI(parent: SparkUI) { private val poolPage = new PoolPage(this) private var _listener: Option[JobProgressListener] = None + def appName = parent.appName + def start() { val conf = if (live) sc.conf else new SparkConf _listener = Some(new JobProgressListener(conf)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index bd33182b70059..3638e6035ba81 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -27,12 +27,13 @@ import org.apache.spark.ui.UIUtils /** Page showing specific pool details */ private[ui] class PoolPage(parent: JobProgressUI) { - private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc private lazy val listener = parent.listener + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val poolName = request.getParameter("poolname") diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 0c55f2ee7e944..0bcbd7461cc5b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -28,10 +28,11 @@ import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: JobProgressUI) { - private val appName = parent.appName private val basePath = parent.basePath private lazy val listener = parent.listener + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val stageId = request.getParameter("id").toInt diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index a7b24ff695214..16996a2da1e72 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -30,7 +30,6 @@ import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[ui] class BlockManagerUI(parent: SparkUI) { - val appName = parent.appName val basePath = parent.basePath private val indexPage = new IndexPage(this) @@ -39,6 +38,8 @@ private[ui] class BlockManagerUI(parent: SparkUI) { lazy val listener = _listener.get + def appName = parent.appName + def start() { _listener = Some(new BlockManagerListener(parent.storageStatusListener)) } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 0fa461e5e9d27..4f6acc30a88c4 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -28,10 +28,11 @@ import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ private[ui] class IndexPage(parent: BlockManagerUI) { - private val appName = parent.appName private val basePath = parent.basePath private lazy val listener = parent.listener + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { val rdds = listener.rddInfoList diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 3f42eba4ece00..75ee9976d7b5f 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -28,10 +28,11 @@ import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ private[ui] class RDDPage(parent: BlockManagerUI) { - private val appName = parent.appName private val basePath = parent.basePath private lazy val listener = parent.listener + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { val rddId = request.getParameter("id").toInt val storageStatusList = listener.storageStatusList diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index b5f2ec6831d26..0080a8b342b05 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -49,7 +49,7 @@ private[spark] class FileLogger( } private val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) - private var fileIndex = 0 + var fileIndex = 0 // Only used if compression is enabled private lazy val compressionCodec = CompressionCodec.createCodec(conf) @@ -57,10 +57,9 @@ private[spark] class FileLogger( // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None - private var writer: Option[PrintWriter] = { - createLogDir() - Some(createWriter()) - } + private var writer: Option[PrintWriter] = None + + createLogDir() /** * Create a logging directory with the given path. @@ -84,8 +83,8 @@ private[spark] class FileLogger( /** * Create a new writer for the file identified by the given path. */ - private def createWriter(): PrintWriter = { - val logPath = logDir + "/" + fileIndex + private def createWriter(fileName: String): PrintWriter = { + val logPath = logDir + "/" + fileName val uri = new URI(logPath) /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). @@ -147,13 +146,17 @@ private[spark] class FileLogger( } /** - * Start a writer for a new file if one does not already exit. + * Start a writer for a new file, closing the existing one if it exists. + * @param fileName Name of the new file, defaulting to the file index if not provided. */ - def start() { - writer.getOrElse { - fileIndex += 1 - writer = Some(createWriter()) + def newFile(fileName: String = "") { + fileIndex += 1 + writer.foreach(_.close()) + val name = fileName match { + case "" => fileIndex.toString + case _ => fileName } + writer = Some(createWriter(name)) } /** diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 19654892bf661..d990fd49ef834 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -62,6 +62,10 @@ private[spark] object JsonProtocol { blockManagerRemovedToJson(blockManagerRemoved) case unpersistRDD: SparkListenerUnpersistRDD => unpersistRDDToJson(unpersistRDD) + case applicationStart: SparkListenerApplicationStart => + applicationStartToJson(applicationStart) + case applicationEnd: SparkListenerApplicationEnd => + applicationEndToJson(applicationEnd) // Not used, but keeps compiler happy case SparkListenerShutdown => JNothing @@ -157,6 +161,18 @@ private[spark] object JsonProtocol { ("RDD ID" -> unpersistRDD.rddId) } + def applicationStartToJson(applicationStart: SparkListenerApplicationStart): JValue = { + ("Event" -> Utils.getFormattedClassName(applicationStart)) ~ + ("App Name" -> applicationStart.appName) ~ + ("Timestamp" -> applicationStart.time) ~ + ("User" -> applicationStart.sparkUser) + } + + def applicationEndToJson(applicationEnd: SparkListenerApplicationEnd): JValue = { + ("Event" -> Utils.getFormattedClassName(applicationEnd)) ~ + ("Timestamp" -> applicationEnd.time) + } + /** ------------------------------------------------------------------- * * JSON serialization methods for classes SparkListenerEvents depend on | @@ -346,6 +362,8 @@ private[spark] object JsonProtocol { val blockManagerAdded = Utils.getFormattedClassName(SparkListenerBlockManagerAdded) val blockManagerRemoved = Utils.getFormattedClassName(SparkListenerBlockManagerRemoved) val unpersistRDD = Utils.getFormattedClassName(SparkListenerUnpersistRDD) + val applicationStart = Utils.getFormattedClassName(SparkListenerApplicationStart) + val applicationEnd = Utils.getFormattedClassName(SparkListenerApplicationEnd) (json \ "Event").extract[String] match { case `stageSubmitted` => stageSubmittedFromJson(json) @@ -359,6 +377,8 @@ private[spark] object JsonProtocol { case `blockManagerAdded` => blockManagerAddedFromJson(json) case `blockManagerRemoved` => blockManagerRemovedFromJson(json) case `unpersistRDD` => unpersistRDDFromJson(json) + case `applicationStart` => applicationStartFromJson(json) + case `applicationEnd` => applicationEndFromJson(json) } } @@ -430,6 +450,17 @@ private[spark] object JsonProtocol { SparkListenerUnpersistRDD((json \ "RDD ID").extract[Int]) } + def applicationStartFromJson(json: JValue): SparkListenerApplicationStart = { + val appName = (json \ "App Name").extract[String] + val time = (json \ "Timestamp").extract[Long] + val sparkUser = (json \ "User").extract[String] + SparkListenerApplicationStart(appName, time, sparkUser) + } + + def applicationEndFromJson(json: JValue): SparkListenerApplicationEnd = { + SparkListenerApplicationEnd((json \ "Timestamp").extract[Long]) + } + /** --------------------------------------------------------------------- * * JSON deserialization methods for classes SparkListenerEvents depend on | diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 59da51f3e0297..166f48ce7342e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -26,7 +26,6 @@ import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.SortedSet import scala.io.Source import scala.reflect.ClassTag @@ -1022,4 +1021,11 @@ private[spark] object Utils extends Logging { def getHadoopFileSystem(path: URI): FileSystem = { FileSystem.get(path, SparkHadoopUtil.get.newConfiguration()) } + + /** + * Return a Hadoop FileSystem with the scheme encoded in the given path. + */ + def getHadoopFileSystem(path: String): FileSystem = { + getHadoopFileSystem(new URI(path)) + } } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index beac656f573b4..8c06a2d9aa4ab 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui.jobs import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, Success} +import org.apache.spark.{LocalSparkContext, SparkConf, Success} import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} import org.apache.spark.scheduler._ import org.apache.spark.util.Utils diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 0342a8aff3c28..f75297a02dc8b 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import java.util.{Properties, UUID} +import java.util.Properties import scala.collection.Map @@ -52,6 +52,8 @@ class JsonProtocolSuite extends FunSuite { val blockManagerRemoved = SparkListenerBlockManagerRemoved( BlockManagerId("Scarce", "to be counted...", 100, 200)) val unpersistRdd = SparkListenerUnpersistRDD(12345) + val applicationStart = SparkListenerApplicationStart("The winner of all", 42L, "Garfield") + val applicationEnd = SparkListenerApplicationEnd(42L) testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) @@ -64,6 +66,8 @@ class JsonProtocolSuite extends FunSuite { testEvent(blockManagerAdded, blockManagerAddedJsonString) testEvent(blockManagerRemoved, blockManagerRemovedJsonString) testEvent(unpersistRdd, unpersistRDDJsonString) + testEvent(applicationStart, applicationStartJsonString) + testEvent(applicationEnd, applicationEndJsonString) } test("Dependent Classes") { @@ -208,7 +212,13 @@ class JsonProtocolSuite extends FunSuite { case (e1: SparkListenerBlockManagerRemoved, e2: SparkListenerBlockManagerRemoved) => assertEquals(e1.blockManagerId, e2.blockManagerId) case (e1: SparkListenerUnpersistRDD, e2: SparkListenerUnpersistRDD) => - assert(e1.rddId === e2.rddId) + assert(e1.rddId == e2.rddId) + case (e1: SparkListenerApplicationStart, e2: SparkListenerApplicationStart) => + assert(e1.appName == e2.appName) + assert(e1.time == e2.time) + assert(e1.sparkUser == e2.sparkUser) + case (e1: SparkListenerApplicationEnd, e2: SparkListenerApplicationEnd) => + assert(e1.time == e2.time) case (SparkListenerShutdown, SparkListenerShutdown) => case _ => fail("Events don't match in types!") } @@ -553,4 +563,14 @@ class JsonProtocolSuite extends FunSuite { {"Event":"SparkListenerUnpersistRDD","RDD ID":12345} """ + private val applicationStartJsonString = + """ + {"Event":"SparkListenerApplicationStart","App Name":"The winner of all","Timestamp":42, + "User":"Garfield"} + """ + + private val applicationEndJsonString = + """ + {"Event":"SparkListenerApplicationEnd","Timestamp":42} + """ } diff --git a/docs/monitoring.md b/docs/monitoring.md index 15bfb041780da..4c91c3a5929bf 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -12,17 +12,77 @@ displays useful information about the application. This includes: * A list of scheduler stages and tasks * A summary of RDD sizes and memory usage -* Information about the running executors * Environmental information. +* Information about the running executors You can access this interface by simply opening `http://:4040` in a web browser. -If multiple SparkContexts are running on the same host, they will bind to succesive ports +If multiple SparkContexts are running on the same host, they will bind to successive ports beginning with 4040 (4041, 4042, etc). -Spark's Standalone Mode cluster manager also has its own -[web UI](spark-standalone.html#monitoring-and-logging). +Note that this information is only available for the duration of the application by default. +To view the web UI after the fact, set `spark.eventLog.enabled` to true before starting the +application. This configures Spark to log Spark events that encode the information displayed +in the UI to persisted storage. -Note that in both of these UIs, the tables are sortable by clicking their headers, +## Viewing After the Fact + +Spark's Standalone Mode cluster manager also has its own +[web UI](spark-standalone.html#monitoring-and-logging). If an application has logged events over +the course of its lifetime, then the Standalone master's web UI will automatically re-render the +application's UI after the application has finished. + +If Spark is run on Mesos or YARN, it is still possible to reconstruct the UI of a finished +application through Spark's history server, provided that the application's event logs exist. +You can start a the history server by executing: + + ./sbin/start-history-server.sh + +The base logging directory must be supplied, and should contain sub-directories that each +represents an application's event logs. This creates a web interface at +`http://:18080` by default. The history server depends on the following variables: + + + + + + + + + + + +
Environment VariableMeaning
SPARK_DAEMON_MEMORYMemory to allocate to the history server. (default: 512m).
SPARK_DAEMON_JAVA_OPTSJVM options for the history server (default: none).
+ +Further, the history server can be configured as follows: + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.history.updateInterval10 + The period, in seconds, at which information displayed by this history server is updated. + Each update checks for any changes made to the event logs in persisted storage. +
spark.history.retainedApplications250 + The number of application UIs to retain. If this cap is exceeded, then the oldest + applications will be removed. +
spark.history.ui.port18080 + The port to which the web interface of the history server binds. +
+ +Note that in all of these UIs, the tables are sortable by clicking their headers, making it easy to identify slow tasks, data skew, etc. # Metrics diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala index 3ebf288130fb6..910b31d209e13 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala @@ -116,14 +116,14 @@ trait SparkILoopInit { } } - def initializeSpark() { + def initializeSpark() { intp.beQuietDuring { command(""" @transient val sc = org.apache.spark.repl.Main.interp.createSparkContext(); """) command("import org.apache.spark.SparkContext._") } - echo("Spark context available as sc.") + echo("Spark context available as sc.") } // code to be executed only after the interpreter is initialized diff --git a/sbin/start-history-server.sh b/sbin/start-history-server.sh new file mode 100755 index 0000000000000..4a90c68763b68 --- /dev/null +++ b/sbin/start-history-server.sh @@ -0,0 +1,37 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# Starts the history server on the machine this script is executed on. +# +# Usage: start-history-server.sh [] +# Example: ./start-history-server.sh --dir /tmp/spark-events --port 18080 +# + +sbin=`dirname "$0"` +sbin=`cd "$sbin"; pwd` + +if [ $# -lt 1 ]; then + echo "Usage: ./start-history-server.sh " + echo "Example: ./start-history-server.sh /tmp/spark-events" + exit +fi + +LOG_DIR=$1 + +"$sbin"/spark-daemon.sh start org.apache.spark.deploy.history.HistoryServer 1 --dir "$LOG_DIR" diff --git a/sbin/stop-history-server.sh b/sbin/stop-history-server.sh new file mode 100755 index 0000000000000..c0034ad641cbe --- /dev/null +++ b/sbin/stop-history-server.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# Stops the history server on the machine this script is executed on. + +sbin=`dirname "$0"` +sbin=`cd "$sbin"; pwd` + +"$sbin"/spark-daemon.sh stop org.apache.spark.deploy.history.HistoryServer 1 From 3bd312940e2f5250edaf3e88d6c23de25bb1d0a9 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Thu, 10 Apr 2014 11:17:41 -0700 Subject: [PATCH 045/641] SPARK-1428: MLlib should convert non-float64 NumPy arrays to float64 instead of complaining Author: Sandeep Closes #356 from techaddict/1428 and squashes the following commits: 3bdf5f6 [Sandeep] SPARK-1428: MLlib should convert non-float64 NumPy arrays to float64 instead of complaining --- python/pyspark/mllib/_common.py | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index 20a0e309d1494..7ef251d24c77e 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -15,8 +15,9 @@ # limitations under the License. # -from numpy import ndarray, copyto, float64, int64, int32, ones, array_equal, array, dot, shape +from numpy import ndarray, copyto, float64, int64, int32, ones, array_equal, array, dot, shape, complex, issubdtype from pyspark import SparkContext, RDD +import numpy as np from pyspark.serializers import Serializer import struct @@ -47,13 +48,22 @@ def _deserialize_byte_array(shape, ba, offset): return ar.copy() def _serialize_double_vector(v): - """Serialize a double vector into a mutually understood format.""" + """Serialize a double vector into a mutually understood format. + + >>> x = array([1,2,3]) + >>> y = _deserialize_double_vector(_serialize_double_vector(x)) + >>> array_equal(y, array([1.0, 2.0, 3.0])) + True + """ if type(v) != ndarray: raise TypeError("_serialize_double_vector called on a %s; " "wanted ndarray" % type(v)) + """complex is only datatype that can't be converted to float64""" + if issubdtype(v.dtype, complex): + raise TypeError("_serialize_double_vector called on a %s; " + "wanted ndarray" % type(v)) if v.dtype != float64: - raise TypeError("_serialize_double_vector called on an ndarray of %s; " - "wanted ndarray of float64" % v.dtype) + v = v.astype(float64) if v.ndim != 1: raise TypeError("_serialize_double_vector called on a %ddarray; " "wanted a 1darray" % v.ndim) From 7b52b66312994d4dbf243eadb6d27eb06350a81f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 10 Apr 2014 14:43:29 -0700 Subject: [PATCH 046/641] Revert "SPARK-1433: Upgrade Mesos dependency to 0.17.0" This reverts commit 12c077d5aa0b76a808a55db625c9677a52bd43f9. --- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 6 ++---- .../scheduler/cluster/mesos/MesosSchedulerBackend.scala | 2 +- docs/_config.yml | 2 +- pom.xml | 6 +++--- project/SparkBuild.scala | 2 +- 5 files changed, 8 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index c478e685641d7..06b041e1fd9a9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -194,12 +194,10 @@ private[spark] class CoarseMesosSchedulerBackend( .addResources(createResource("cpus", cpusToUse)) .addResources(createResource("mem", sc.executorMemory)) .build() - d.launchTasks(Collections.singletonList(offer.getId), - Collections.singletonList(task), - filters) + d.launchTasks(offer.getId, Collections.singletonList(task), filters) } else { // Filter it out - d.declineOffer(offer.getId, filters) + d.launchTasks(offer.getId, Collections.emptyList[MesosTaskInfo](), filters) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index f878ae338fc95..dfdcafe19fb93 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -223,7 +223,7 @@ private[spark] class MesosSchedulerBackend( // Reply to the offers val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? for (i <- 0 until offers.size) { - d.launchTasks(Collections.singletonList(offers(i).getId), mesosTasks(i), filters) + d.launchTasks(offers(i).getId, mesosTasks(i), filters) } } } finally { diff --git a/docs/_config.yml b/docs/_config.yml index bd5ed6c9220d2..d585b8c5ea763 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -7,6 +7,6 @@ SPARK_VERSION: 1.0.0-SNAPSHOT SPARK_VERSION_SHORT: 1.0.0 SCALA_BINARY_VERSION: "2.10" SCALA_VERSION: "2.10.4" -MESOS_VERSION: 0.17.0 +MESOS_VERSION: 0.13.0 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/pom.xml b/pom.xml index 11511bcb9da52..c03bb35c99442 100644 --- a/pom.xml +++ b/pom.xml @@ -112,7 +112,7 @@ 2.10.4 2.10 - 0.17.0 + 0.13.0 org.spark-project.akka 2.2.3-shaded-protobuf 1.7.5 @@ -848,7 +848,7 @@ - + hadoop-provided @@ -893,6 +893,6 @@ - + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 694f90a83ab67..21163760e6277 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -333,7 +333,7 @@ object SparkBuild extends Build { "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", - "org.apache.mesos" % "mesos" % "0.17.0", + "org.apache.mesos" % "mesos" % "0.13.0", "commons-net" % "commons-net" % "2.2", "net.java.dev.jets3t" % "jets3t" % "0.7.1" excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", From f0466625200842f3cc486e9aa1caa417586be533 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Thu, 10 Apr 2014 14:59:58 -0700 Subject: [PATCH 047/641] Update tuning.md http://stackoverflow.com/questions/9699071/what-is-the-javas-internal-represention-for-string-modified-utf-8-utf-16 Author: Andrew Ash Closes #384 from ash211/patch-2 and squashes the following commits: da1b0be [Andrew Ash] Update tuning.md --- docs/tuning.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/tuning.md b/docs/tuning.md index 093df3187a789..cc069f0e84b9c 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -90,9 +90,10 @@ than the "raw" data inside their fields. This is due to several reasons: * Each distinct Java object has an "object header", which is about 16 bytes and contains information such as a pointer to its class. For an object with very little data in it (say one `Int` field), this can be bigger than the data. -* Java Strings have about 40 bytes of overhead over the raw string data (since they store it in an +* Java `String`s have about 40 bytes of overhead over the raw string data (since they store it in an array of `Char`s and keep extra data such as the length), and store each character - as *two* bytes due to Unicode. Thus a 10-character string can easily consume 60 bytes. + as *two* bytes due to `String`'s internal usage of UTF-16 encoding. Thus a 10-character string can + easily consume 60 bytes. * Common collection classes, such as `HashMap` and `LinkedList`, use linked data structures, where there is a "wrapper" object for each entry (e.g. `Map.Entry`). This object not only has a header, but also pointers (typically 8 bytes each) to the next object in the list. From 930b70f0523e96fe01c1317ef7fad1b76b36d4d9 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Thu, 10 Apr 2014 15:04:13 -0700 Subject: [PATCH 048/641] Remove Unnecessary Whitespace's stack these together in a commit else they show up chunk by chunk in different commits. Author: Sandeep Closes #380 from techaddict/white_space and squashes the following commits: b58f294 [Sandeep] Remove Unnecessary Whitespace's --- .../org/apache/spark/bagel/BagelSuite.scala | 4 +- .../api/java/function/FlatMapFunction.java | 2 +- .../api/java/function/FlatMapFunction2.java | 2 +- .../org/apache/spark/HttpFileServer.scala | 14 +- .../scala/org/apache/spark/HttpServer.scala | 6 +- .../scala/org/apache/spark/Partition.scala | 2 +- .../org/apache/spark/SecurityManager.scala | 88 +++---- .../org/apache/spark/SparkException.scala | 2 +- .../org/apache/spark/SparkHadoopWriter.scala | 20 +- .../org/apache/spark/SparkSaslClient.scala | 10 +- .../org/apache/spark/SparkSaslServer.scala | 6 +- .../scala/org/apache/spark/TestUtils.scala | 2 +- .../spark/broadcast/TorrentBroadcast.scala | 2 +- .../apache/spark/deploy/ClientArguments.scala | 2 +- .../spark/deploy/worker/WorkerArguments.scala | 8 +- .../spark/deploy/worker/ui/IndexPage.scala | 2 +- .../CoarseGrainedExecutorBackend.scala | 2 +- .../spark/executor/ExecutorExitCode.scala | 8 +- .../executor/ExecutorURLClassLoader.scala | 2 +- .../apache/spark/metrics/sink/CsvSink.scala | 2 +- .../org/apache/spark/network/Connection.scala | 8 +- .../apache/spark/network/ConnectionId.scala | 6 +- .../spark/network/ConnectionManager.scala | 28 +-- .../spark/network/ConnectionManagerTest.scala | 24 +- .../apache/spark/network/ReceiverTest.scala | 2 +- .../spark/network/SecurityMessage.scala | 48 ++-- .../spark/network/netty/FileHeader.scala | 4 +- .../apache/spark/partial/PartialResult.scala | 4 +- .../apache/spark/rdd/DoubleRDDFunctions.scala | 8 +- .../spark/rdd/PartitionerAwareUnionRDD.scala | 2 +- .../spark/scheduler/DAGSchedulerEvent.scala | 2 +- .../spark/scheduler/LiveListenerBus.scala | 214 +++++++++--------- .../spark/storage/BlockFetcherIterator.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 4 +- .../apache/spark/storage/BlockMessage.scala | 22 +- .../spark/storage/BlockMessageArray.scala | 26 +-- .../org/apache/spark/ui/JettyUtils.scala | 4 +- .../scala/org/apache/spark/ui/UIUtils.scala | 2 +- .../apache/spark/util/ClosureCleaner.scala | 22 +- .../org/apache/spark/util/JsonProtocol.scala | 2 +- .../org/apache/spark/util/NextIterator.scala | 4 +- .../org/apache/spark/util/StatCounter.scala | 4 +- .../scala/org/apache/spark/util/Vector.scala | 2 +- .../spark/util/random/XORShiftRandom.scala | 16 +- .../org/apache/spark/AkkaUtilsSuite.scala | 20 +- .../scala/org/apache/spark/DriverSuite.scala | 2 +- .../org/apache/spark/FileServerSuite.scala | 4 +- .../scala/org/apache/spark/FileSuite.scala | 2 +- .../deploy/worker/WorkerWatcherSuite.scala | 2 +- .../WholeTextFileRecordReaderSuite.scala | 2 +- .../rdd/ParallelCollectionSplitSuite.scala | 26 +-- .../spark/scheduler/SparkListenerSuite.scala | 4 +- .../scheduler/TaskSchedulerImplSuite.scala | 2 +- .../scala/org/apache/spark/ui/UISuite.scala | 4 +- .../spark/util/ClosureCleanerSuite.scala | 2 +- .../apache/spark/util/NextIteratorSuite.scala | 4 +- .../util/random/XORShiftRandomSuite.scala | 20 +- .../streaming/mqtt/MQTTInputDStream.scala | 16 +- .../twitter/TwitterInputDStream.scala | 6 +- .../org/apache/spark/graphx/GraphOps.scala | 2 +- .../apache/spark/graphx/GraphOpsSuite.scala | 2 +- .../spark/mllib/optimization/Optimizer.scala | 2 +- .../GeneralizedLinearAlgorithm.scala | 4 +- .../spark/repl/ExecutorClassLoader.scala | 4 +- .../org/apache/spark/repl/SparkImports.scala | 2 +- .../spark/sql/catalyst/expressions/Cast.scala | 6 +- .../sql/catalyst/expressions/Expression.scala | 12 +- .../expressions/stringOperations.scala | 28 +-- .../spark/sql/catalyst/types/dataTypes.scala | 4 +- .../ExpressionEvaluationSuite.scala | 10 +- .../sql/ScalaReflectionRelationSuite.scala | 2 +- .../apache/spark/streaming/Checkpoint.scala | 14 +- .../org/apache/spark/streaming/Interval.scala | 8 +- .../org/apache/spark/streaming/Time.scala | 4 +- .../dstream/DStreamCheckpointData.scala | 2 +- .../streaming/dstream/FileInputDStream.scala | 2 +- .../streaming/dstream/QueueInputDStream.scala | 8 +- .../streaming/receivers/ActorReceiver.scala | 2 +- .../apache/spark/streaming/util/Clock.scala | 26 +-- .../spark/streaming/util/RawTextHelper.scala | 18 +- .../spark/streaming/util/RecurringTimer.scala | 8 +- .../apache/spark/streaming/JavaAPISuite.java | 2 +- 82 files changed, 467 insertions(+), 467 deletions(-) diff --git a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala index 9c37fadb78d2f..69144e3e657bf 100644 --- a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala @@ -28,9 +28,9 @@ class TestVertex(val active: Boolean, val age: Int) extends Vertex with Serializ class TestMessage(val targetId: String) extends Message[String] with Serializable class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeouts { - + var sc: SparkContext = _ - + after { if (sc != null) { sc.stop() diff --git a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java index fa75842047c6a..23f5fdd43631b 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java @@ -24,4 +24,4 @@ */ public interface FlatMapFunction extends Serializable { public Iterable call(T t) throws Exception; -} \ No newline at end of file +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java index d1fdec072443d..c48e92f535ff5 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java +++ b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java @@ -24,4 +24,4 @@ */ public interface FlatMapFunction2 extends Serializable { public Iterable call(T1 t1, T2 t2) throws Exception; -} \ No newline at end of file +} diff --git a/core/src/main/scala/org/apache/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala index 3d7692ea8a49e..a6e300d345786 100644 --- a/core/src/main/scala/org/apache/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -24,13 +24,13 @@ import com.google.common.io.Files import org.apache.spark.util.Utils private[spark] class HttpFileServer(securityManager: SecurityManager) extends Logging { - + var baseDir : File = null var fileDir : File = null var jarDir : File = null var httpServer : HttpServer = null var serverUri : String = null - + def initialize() { baseDir = Utils.createTempDir() fileDir = new File(baseDir, "files") @@ -43,24 +43,24 @@ private[spark] class HttpFileServer(securityManager: SecurityManager) extends Lo serverUri = httpServer.uri logDebug("HTTP file server started at: " + serverUri) } - + def stop() { httpServer.stop() } - + def addFile(file: File) : String = { addFileToDir(file, fileDir) serverUri + "/files/" + file.getName } - + def addJar(file: File) : String = { addFileToDir(file, jarDir) serverUri + "/jars/" + file.getName } - + def addFileToDir(file: File, dir: File) : String = { Files.copy(file, new File(dir, file.getName)) dir + "/" + file.getName } - + } diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index cb5df25fa48df..7e9b517f901a2 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -83,19 +83,19 @@ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityMan } } - /** + /** * Setup Jetty to the HashLoginService using a single user with our * shared secret. Configure it to use DIGEST-MD5 authentication so that the password * isn't passed in plaintext. */ private def setupSecurityHandler(securityMgr: SecurityManager): ConstraintSecurityHandler = { val constraint = new Constraint() - // use DIGEST-MD5 as the authentication mechanism + // use DIGEST-MD5 as the authentication mechanism constraint.setName(Constraint.__DIGEST_AUTH) constraint.setRoles(Array("user")) constraint.setAuthenticate(true) constraint.setDataConstraint(Constraint.DC_NONE) - + val cm = new ConstraintMapping() cm.setConstraint(constraint) cm.setPathSpec("/*") diff --git a/core/src/main/scala/org/apache/spark/Partition.scala b/core/src/main/scala/org/apache/spark/Partition.scala index 87914a061f5d7..27892dbd2a0bc 100644 --- a/core/src/main/scala/org/apache/spark/Partition.scala +++ b/core/src/main/scala/org/apache/spark/Partition.scala @@ -25,7 +25,7 @@ trait Partition extends Serializable { * Get the split's index within its parent RDD */ def index: Int - + // A better default implementation of HashCode override def hashCode(): Int = index } diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 2237ee3bb7aad..b52f2d4f416b2 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -25,93 +25,93 @@ import org.apache.hadoop.io.Text import org.apache.spark.deploy.SparkHadoopUtil -/** - * Spark class responsible for security. - * +/** + * Spark class responsible for security. + * * In general this class should be instantiated by the SparkEnv and most components - * should access it from that. There are some cases where the SparkEnv hasn't been + * should access it from that. There are some cases where the SparkEnv hasn't been * initialized yet and this class must be instantiated directly. - * + * * Spark currently supports authentication via a shared secret. * Authentication can be configured to be on via the 'spark.authenticate' configuration - * parameter. This parameter controls whether the Spark communication protocols do + * parameter. This parameter controls whether the Spark communication protocols do * authentication using the shared secret. This authentication is a basic handshake to * make sure both sides have the same shared secret and are allowed to communicate. - * If the shared secret is not identical they will not be allowed to communicate. - * - * The Spark UI can also be secured by using javax servlet filters. A user may want to - * secure the UI if it has data that other users should not be allowed to see. The javax - * servlet filter specified by the user can authenticate the user and then once the user - * is logged in, Spark can compare that user versus the view acls to make sure they are - * authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls' + * If the shared secret is not identical they will not be allowed to communicate. + * + * The Spark UI can also be secured by using javax servlet filters. A user may want to + * secure the UI if it has data that other users should not be allowed to see. The javax + * servlet filter specified by the user can authenticate the user and then once the user + * is logged in, Spark can compare that user versus the view acls to make sure they are + * authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls' * control the behavior of the acls. Note that the person who started the application * always has view access to the UI. * * Spark does not currently support encryption after authentication. - * + * * At this point spark has multiple communication protocols that need to be secured and * different underlying mechanisms are used depending on the protocol: * - * - Akka -> The only option here is to use the Akka Remote secure-cookie functionality. - * Akka remoting allows you to specify a secure cookie that will be exchanged - * and ensured to be identical in the connection handshake between the client - * and the server. If they are not identical then the client will be refused - * to connect to the server. There is no control of the underlying - * authentication mechanism so its not clear if the password is passed in + * - Akka -> The only option here is to use the Akka Remote secure-cookie functionality. + * Akka remoting allows you to specify a secure cookie that will be exchanged + * and ensured to be identical in the connection handshake between the client + * and the server. If they are not identical then the client will be refused + * to connect to the server. There is no control of the underlying + * authentication mechanism so its not clear if the password is passed in * plaintext or uses DIGEST-MD5 or some other mechanism. * Akka also has an option to turn on SSL, this option is not currently supported * but we could add a configuration option in the future. - * - * - HTTP for broadcast and file server (via HttpServer) -> Spark currently uses Jetty - * for the HttpServer. Jetty supports multiple authentication mechanisms - - * Basic, Digest, Form, Spengo, etc. It also supports multiple different login + * + * - HTTP for broadcast and file server (via HttpServer) -> Spark currently uses Jetty + * for the HttpServer. Jetty supports multiple authentication mechanisms - + * Basic, Digest, Form, Spengo, etc. It also supports multiple different login * services - Hash, JAAS, Spnego, JDBC, etc. Spark currently uses the HashLoginService - * to authenticate using DIGEST-MD5 via a single user and the shared secret. + * to authenticate using DIGEST-MD5 via a single user and the shared secret. * Since we are using DIGEST-MD5, the shared secret is not passed on the wire * in plaintext. * We currently do not support SSL (https), but Jetty can be configured to use it * so we could add a configuration option for this in the future. - * + * * The Spark HttpServer installs the HashLoginServer and configures it to DIGEST-MD5. - * Any clients must specify the user and password. There is a default + * Any clients must specify the user and password. There is a default * Authenticator installed in the SecurityManager to how it does the authentication * and in this case gets the user name and password from the request. * - * - ConnectionManager -> The Spark ConnectionManager uses java nio to asynchronously - * exchange messages. For this we use the Java SASL - * (Simple Authentication and Security Layer) API and again use DIGEST-MD5 + * - ConnectionManager -> The Spark ConnectionManager uses java nio to asynchronously + * exchange messages. For this we use the Java SASL + * (Simple Authentication and Security Layer) API and again use DIGEST-MD5 * as the authentication mechanism. This means the shared secret is not passed * over the wire in plaintext. * Note that SASL is pluggable as to what mechanism it uses. We currently use * DIGEST-MD5 but this could be changed to use Kerberos or other in the future. * Spark currently supports "auth" for the quality of protection, which means * the connection is not supporting integrity or privacy protection (encryption) - * after authentication. SASL also supports "auth-int" and "auth-conf" which + * after authentication. SASL also supports "auth-int" and "auth-conf" which * SPARK could be support in the future to allow the user to specify the quality - * of protection they want. If we support those, the messages will also have to + * of protection they want. If we support those, the messages will also have to * be wrapped and unwrapped via the SaslServer/SaslClient.wrap/unwrap API's. - * - * Since the connectionManager does asynchronous messages passing, the SASL + * + * Since the connectionManager does asynchronous messages passing, the SASL * authentication is a bit more complex. A ConnectionManager can be both a client * and a Server, so for a particular connection is has to determine what to do. - * A ConnectionId was added to be able to track connections and is used to + * A ConnectionId was added to be able to track connections and is used to * match up incoming messages with connections waiting for authentication. * If its acting as a client and trying to send a message to another ConnectionManager, * it blocks the thread calling sendMessage until the SASL negotiation has occurred. * The ConnectionManager tracks all the sendingConnections using the ConnectionId * and waits for the response from the server and does the handshake. * - * - HTTP for the Spark UI -> the UI was changed to use servlets so that javax servlet filters + * - HTTP for the Spark UI -> the UI was changed to use servlets so that javax servlet filters * can be used. Yarn requires a specific AmIpFilter be installed for security to work * properly. For non-Yarn deployments, users can write a filter to go through a * companies normal login service. If an authentication filter is in place then the * SparkUI can be configured to check the logged in user against the list of users who * have view acls to see if that user is authorized. - * The filters can also be used for many different purposes. For instance filters + * The filters can also be used for many different purposes. For instance filters * could be used for logging, encryption, or compression. - * + * * The exact mechanisms used to generate/distributed the shared secret is deployment specific. - * + * * For Yarn deployments, the secret is automatically generated using the Akka remote * Crypt.generateSecureCookie() API. The secret is placed in the Hadoop UGI which gets passed * around via the Hadoop RPC mechanism. Hadoop RPC can be configured to support different levels @@ -121,7 +121,7 @@ import org.apache.spark.deploy.SparkHadoopUtil * to reduce the possibility of web based attacks through YARN. Hadoop can be configured to use * filters to do authentication. That authentication then happens via the ResourceManager Proxy * and Spark will use that to do authorization against the view acls. - * + * * For other Spark deployments, the shared secret must be specified via the * spark.authenticate.secret config. * All the nodes (Master and Workers) and the applications need to have the same shared secret. @@ -152,7 +152,7 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { " are ui acls enabled: " + uiAclsOn + " users with view permissions: " + viewAcls.toString()) // Set our own authenticator to properly negotiate user/password for HTTP connections. - // This is needed by the HTTP client fetching from the HttpServer. Put here so its + // This is needed by the HTTP client fetching from the HttpServer. Put here so its // only set once. if (authOn) { Authenticator.setDefault( @@ -214,12 +214,12 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { def uiAclsEnabled(): Boolean = uiAclsOn /** - * Checks the given user against the view acl list to see if they have + * Checks the given user against the view acl list to see if they have * authorization to view the UI. If the UI acls must are disabled * via spark.ui.acls.enable, all users have view access. - * + * * @param user to see if is authorized - * @return true is the user has permission, otherwise false + * @return true is the user has permission, otherwise false */ def checkUIViewPermissions(user: String): Boolean = { if (uiAclsEnabled() && (user != null) && (!viewAcls.contains(user))) false else true diff --git a/core/src/main/scala/org/apache/spark/SparkException.scala b/core/src/main/scala/org/apache/spark/SparkException.scala index d34e47e8cac22..4351ed74b67fc 100644 --- a/core/src/main/scala/org/apache/spark/SparkException.scala +++ b/core/src/main/scala/org/apache/spark/SparkException.scala @@ -20,5 +20,5 @@ package org.apache.spark class SparkException(message: String, cause: Throwable) extends Exception(message, cause) { - def this(message: String) = this(message, null) + def this(message: String) = this(message, null) } diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index b92ea01a877f7..f6703986bdf11 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -42,7 +42,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) private val now = new Date() private val conf = new SerializableWritable(jobConf) - + private var jobID = 0 private var splitID = 0 private var attemptID = 0 @@ -58,8 +58,8 @@ class SparkHadoopWriter(@transient jobConf: JobConf) def preSetup() { setIDs(0, 0, 0) HadoopRDD.addLocalConfiguration("", 0, 0, 0, conf.value) - - val jCtxt = getJobContext() + + val jCtxt = getJobContext() getOutputCommitter().setupJob(jCtxt) } @@ -74,7 +74,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) val numfmt = NumberFormat.getInstance() numfmt.setMinimumIntegerDigits(5) numfmt.setGroupingUsed(false) - + val outputName = "part-" + numfmt.format(splitID) val path = FileOutputFormat.getOutputPath(conf.value) val fs: FileSystem = { @@ -85,7 +85,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) } } - getOutputCommitter().setupTask(getTaskContext()) + getOutputCommitter().setupTask(getTaskContext()) writer = getOutputFormat().getRecordWriter(fs, conf.value, outputName, Reporter.NULL) } @@ -103,18 +103,18 @@ class SparkHadoopWriter(@transient jobConf: JobConf) def commit() { val taCtxt = getTaskContext() - val cmtr = getOutputCommitter() + val cmtr = getOutputCommitter() if (cmtr.needsTaskCommit(taCtxt)) { try { cmtr.commitTask(taCtxt) logInfo (taID + ": Committed") } catch { - case e: IOException => { + case e: IOException => { logError("Error committing the output of task: " + taID.value, e) cmtr.abortTask(taCtxt) throw e } - } + } } else { logWarning ("No need to commit output of task: " + taID.value) } @@ -144,7 +144,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) } private def getJobContext(): JobContext = { - if (jobContext == null) { + if (jobContext == null) { jobContext = newJobContext(conf.value, jID.value) } jobContext @@ -175,7 +175,7 @@ object SparkHadoopWriter { val jobtrackerID = formatter.format(time) new JobID(jobtrackerID, id) } - + def createPathFromString(path: String, conf: JobConf): Path = { if (path == null) { throw new IllegalArgumentException("Output path is null") diff --git a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala index a2a871cbd3c31..5b14c4291d91a 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala @@ -44,12 +44,12 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg * configurable in the future. */ private var saslClient: SaslClient = Sasl.createSaslClient(Array[String](SparkSaslServer.DIGEST), - null, null, SparkSaslServer.SASL_DEFAULT_REALM, SparkSaslServer.SASL_PROPS, + null, null, SparkSaslServer.SASL_DEFAULT_REALM, SparkSaslServer.SASL_PROPS, new SparkSaslClientCallbackHandler(securityMgr)) /** * Used to initiate SASL handshake with server. - * @return response to challenge if needed + * @return response to challenge if needed */ def firstToken(): Array[Byte] = { synchronized { @@ -86,7 +86,7 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg } /** - * Disposes of any system resources or security-sensitive information the + * Disposes of any system resources or security-sensitive information the * SaslClient might be using. */ def dispose() { @@ -110,7 +110,7 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg private class SparkSaslClientCallbackHandler(securityMgr: SecurityManager) extends CallbackHandler { - private val userName: String = + private val userName: String = SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes()) private val secretKey = securityMgr.getSecretKey() private val userPassword: Array[Char] = @@ -138,7 +138,7 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg rc.setText(rc.getDefaultText()) } case cb: RealmChoiceCallback => {} - case cb: Callback => throw + case cb: Callback => throw new UnsupportedCallbackException(cb, "handle: Unrecognized SASL client callback") } } diff --git a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala index 11fcb2ae3a5c5..6161a6fb7ae85 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala @@ -64,7 +64,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi } /** - * Disposes of any system resources or security-sensitive information the + * Disposes of any system resources or security-sensitive information the * SaslServer might be using. */ def dispose() { @@ -88,7 +88,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi private class SparkSaslDigestCallbackHandler(securityMgr: SecurityManager) extends CallbackHandler { - private val userName: String = + private val userName: String = SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes()) override def handle(callbacks: Array[Callback]) { @@ -123,7 +123,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi ac.setAuthorizedID(authzid) } } - case cb: Callback => throw + case cb: Callback => throw new UnsupportedCallbackException(cb, "handle: Unrecognized SASL DIGEST-MD5 Callback") } } diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 4597595a838e3..f3f59e47c3e98 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -31,7 +31,7 @@ import com.google.common.io.Files * projects. * * TODO: See if we can move this to the test codebase by specifying - * test dependencies between projects. + * test dependencies between projects. */ private[spark] object TestUtils { diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 2b32546c6854d..2659274c5e98e 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -158,7 +158,7 @@ private[spark] class TorrentBroadcast[T](@transient var value_ : T, isLocal: Boo } def receiveBroadcast(): Boolean = { - // Receive meta-info about the size of broadcast data, + // Receive meta-info about the size of broadcast data, // the number of chunks it is divided into, etc. val metaId = BroadcastBlockId(id, "meta") var attemptId = 10 diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index c07838f798799..5da9615c9e9af 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -43,7 +43,7 @@ private[spark] class ClientArguments(args: Array[String]) { // kill parameters var driverId: String = "" - + parse(args.toList) def parse(args: List[String]): Unit = args match { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index d35d5be73ff97..3836bf219ed3e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -32,8 +32,8 @@ private[spark] class WorkerArguments(args: Array[String]) { var memory = inferDefaultMemory() var masters: Array[String] = null var workDir: String = null - - // Check for settings in environment variables + + // Check for settings in environment variables if (System.getenv("SPARK_WORKER_PORT") != null) { port = System.getenv("SPARK_WORKER_PORT").toInt } @@ -49,7 +49,7 @@ private[spark] class WorkerArguments(args: Array[String]) { if (System.getenv("SPARK_WORKER_DIR") != null) { workDir = System.getenv("SPARK_WORKER_DIR") } - + parse(args.toList) def parse(args: List[String]): Unit = args match { @@ -78,7 +78,7 @@ private[spark] class WorkerArguments(args: Array[String]) { case ("--work-dir" | "-d") :: value :: tail => workDir = value parse(tail) - + case "--webui-port" :: IntParam(value) :: tail => webUiPort = value parse(tail) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala index 85200ab0e102d..49c1009cac2bf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala @@ -137,7 +137,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { .format(executor.appId, executor.execId)}>stdout stderr - + } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 16887d8892b31..6327ac01663f6 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -53,7 +53,7 @@ private[spark] class CoarseGrainedExecutorBackend( case RegisteredExecutor(sparkProperties) => logInfo("Successfully registered with driver") // Make this host instead of hostPort ? - executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties, + executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties, false) case RegisterExecutorFailed(message) => diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala index ceff3a067d72a..38be2c58b333f 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala @@ -34,7 +34,7 @@ object ExecutorExitCode { logging the exception. */ val UNCAUGHT_EXCEPTION_TWICE = 51 - /** The default uncaught exception handler was reached, and the uncaught exception was an + /** The default uncaught exception handler was reached, and the uncaught exception was an OutOfMemoryError. */ val OOM = 52 @@ -43,10 +43,10 @@ object ExecutorExitCode { /** TachyonStore failed to initialize after many attempts. */ val TACHYON_STORE_FAILED_TO_INITIALIZE = 54 - + /** TachyonStore failed to create a local temporary directory after many attempts. */ val TACHYON_STORE_FAILED_TO_CREATE_DIR = 55 - + def explainExitCode(exitCode: Int): String = { exitCode match { case UNCAUGHT_EXCEPTION => "Uncaught exception" @@ -57,7 +57,7 @@ object ExecutorExitCode { case TACHYON_STORE_FAILED_TO_INITIALIZE => "TachyonStore failed to initialize." case TACHYON_STORE_FAILED_TO_CREATE_DIR => "TachyonStore failed to create a local temporary directory." - case _ => + case _ => "Unknown executor exit code (" + exitCode + ")" + ( if (exitCode > 128) { " (died from signal " + (exitCode - 128) + "?)" diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala index 208e77073fd03..218ed7b5d2d39 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala @@ -38,7 +38,7 @@ private[spark] class ChildExecutorURLClassLoader(urls: Array[URL], parent: Class override def addURL(url: URL) { super.addURL(url) } - override def findClass(name: String): Class[_] = { + override def findClass(name: String): Class[_] = { super.findClass(name) } } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala index 42c1200926fea..542dce65366b2 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala @@ -45,7 +45,7 @@ private[spark] class CsvSink(val property: Properties, val registry: MetricRegis case Some(s) => TimeUnit.valueOf(s.toUpperCase()) case None => TimeUnit.valueOf(CSV_DEFAULT_UNIT) } - + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) val pollDir = Option(property.getProperty(CSV_KEY_DIR)) match { diff --git a/core/src/main/scala/org/apache/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/Connection.scala index 2f7576c53b482..3ffaaab23d0f5 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/Connection.scala @@ -248,14 +248,14 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, } } - // outbox is used as a lock - ensure that it is always used as a leaf (since methods which + // outbox is used as a lock - ensure that it is always used as a leaf (since methods which // lock it are invoked in context of other locks) private val outbox = new Outbox() /* - This is orthogonal to whether we have pending bytes to write or not - and satisfies a slightly - different purpose. This flag is to see if we need to force reregister for write even when we + This is orthogonal to whether we have pending bytes to write or not - and satisfies a slightly + different purpose. This flag is to see if we need to force reregister for write even when we do not have any pending bytes to write to socket. - This can happen due to a race between adding pending buffers, and checking for existing of + This can happen due to a race between adding pending buffers, and checking for existing of data as detailed in https://github.com/mesos/spark/pull/791 */ private var needForceReregister = false diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionId.scala b/core/src/main/scala/org/apache/spark/network/ConnectionId.scala index ffaab677d411a..d579c165a1917 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionId.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionId.scala @@ -18,7 +18,7 @@ package org.apache.spark.network private[spark] case class ConnectionId(connectionManagerId: ConnectionManagerId, uniqId: Int) { - override def toString = connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId + override def toString = connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId } private[spark] object ConnectionId { @@ -26,9 +26,9 @@ private[spark] object ConnectionId { def createConnectionIdFromString(connectionIdString: String): ConnectionId = { val res = connectionIdString.split("_").map(_.trim()) if (res.size != 3) { - throw new Exception("Error converting ConnectionId string: " + connectionIdString + + throw new Exception("Error converting ConnectionId string: " + connectionIdString + " to a ConnectionId Object") } new ConnectionId(new ConnectionManagerId(res(0), res(1).toInt), res(2).toInt) - } + } } diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index bdf586351ac14..cfee41c61362e 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -79,7 +79,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, private val serverChannel = ServerSocketChannel.open() // used to track the SendingConnections waiting to do SASL negotiation - private val connectionsAwaitingSasl = new HashMap[ConnectionId, SendingConnection] + private val connectionsAwaitingSasl = new HashMap[ConnectionId, SendingConnection] with SynchronizedMap[ConnectionId, SendingConnection] private val connectionsByKey = new HashMap[SelectionKey, Connection] with SynchronizedMap[SelectionKey, Connection] @@ -141,7 +141,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } finally { writeRunnableStarted.synchronized { writeRunnableStarted -= key - val needReregister = register || conn.resetForceReregister() + val needReregister = register || conn.resetForceReregister() if (needReregister && conn.changeInterestForWrite()) { conn.registerInterest() } @@ -509,7 +509,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, private def handleClientAuthentication( waitingConn: SendingConnection, - securityMsg: SecurityMessage, + securityMsg: SecurityMessage, connectionId : ConnectionId) { if (waitingConn.isSaslComplete()) { logDebug("Client sasl completed for id: " + waitingConn.connectionId) @@ -530,7 +530,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } return } - var securityMsgResp = SecurityMessage.fromResponse(replyToken, + var securityMsgResp = SecurityMessage.fromResponse(replyToken, securityMsg.getConnectionId.toString()) var message = securityMsgResp.toBufferMessage if (message == null) throw new Exception("Error creating security message") @@ -546,7 +546,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } private def handleServerAuthentication( - connection: Connection, + connection: Connection, securityMsg: SecurityMessage, connectionId: ConnectionId) { if (!connection.isSaslComplete()) { @@ -561,7 +561,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } replyToken = connection.sparkSaslServer.response(securityMsg.getToken) if (connection.isSaslComplete()) { - logDebug("Server sasl completed: " + connection.connectionId) + logDebug("Server sasl completed: " + connection.connectionId) } else { logDebug("Server sasl not completed: " + connection.connectionId) } @@ -571,7 +571,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, var message = securityMsgResp.toBufferMessage if (message == null) throw new Exception("Error creating security Message") sendSecurityMessage(connection.getRemoteConnectionManagerId(), message) - } + } } catch { case e: Exception => { logError("Error in server auth negotiation: " + e) @@ -581,7 +581,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } } else { - logDebug("connection already established for this connection id: " + connection.connectionId) + logDebug("connection already established for this connection id: " + connection.connectionId) } } @@ -609,8 +609,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, return true } else { if (!conn.isSaslComplete()) { - // We could handle this better and tell the client we need to do authentication - // negotiation, but for now just ignore them. + // We could handle this better and tell the client we need to do authentication + // negotiation, but for now just ignore them. logError("message sent that is not security negotiation message on connection " + "not authenticated yet, ignoring it!!") return true @@ -709,11 +709,11 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } } else { - logDebug("Sasl already established ") + logDebug("Sasl already established ") } } - // allow us to add messages to the inbox for doing sasl negotiating + // allow us to add messages to the inbox for doing sasl negotiating private def sendSecurityMessage(connManagerId: ConnectionManagerId, message: Message) { def startNewConnection(): SendingConnection = { val inetSocketAddress = new InetSocketAddress(connManagerId.host, connManagerId.port) @@ -772,7 +772,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, if (((clock.getTime() - startTime) >= (authTimeout * 1000)) && (!connection.isSaslComplete())) { // took to long to authenticate the connection, something probably went wrong - throw new Exception("Took to long for authentication to " + connectionManagerId + + throw new Exception("Took to long for authentication to " + connectionManagerId + ", waited " + authTimeout + "seconds, failing.") } } @@ -794,7 +794,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } case None => { - logError("no messageStatus for failed message id: " + message.id) + logError("no messageStatus for failed message id: " + message.id) } } } diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala index 9d9b9dbdd5331..4894ecd41f6eb 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala @@ -37,11 +37,11 @@ private[spark] object ConnectionManagerTest extends Logging{ "[size of msg in MB (integer)] [count] [await time in seconds)] ") System.exit(1) } - + if (args(0).startsWith("local")) { println("This runs only on a mesos cluster") } - + val sc = new SparkContext(args(0), "ConnectionManagerTest") val slavesFile = Source.fromFile(args(1)) val slaves = slavesFile.mkString.split("\n") @@ -50,7 +50,7 @@ private[spark] object ConnectionManagerTest extends Logging{ /* println("Slaves") */ /* slaves.foreach(println) */ val tasknum = if (args.length > 2) args(2).toInt else slaves.length - val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 + val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 val count = if (args.length > 4) args(4).toInt else 3 val awaitTime = (if (args.length > 5) args(5).toInt else 600 ).second println("Running " + count + " rounds of test: " + "parallel tasks = " + tasknum + ", " + @@ -64,16 +64,16 @@ private[spark] object ConnectionManagerTest extends Logging{ (0 until count).foreach(i => { val resultStrs = sc.parallelize(0 until tasknum, tasknum).map(i => { val connManager = SparkEnv.get.connectionManager - val thisConnManagerId = connManager.id - connManager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + val thisConnManagerId = connManager.id + connManager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { logInfo("Received [" + msg + "] from [" + id + "]") None }) val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) buffer.flip - - val startTime = System.currentTimeMillis + + val startTime = System.currentTimeMillis val futures = slaveConnManagerIds.filter(_ != thisConnManagerId).map{ slaveConnManagerId => { val bufferMessage = Message.createBufferMessage(buffer.duplicate) @@ -84,7 +84,7 @@ private[spark] object ConnectionManagerTest extends Logging{ val results = futures.map(f => Await.result(f, awaitTime)) val finishTime = System.currentTimeMillis Thread.sleep(5000) - + val mb = size * results.size / 1024.0 / 1024.0 val ms = finishTime - startTime val resultStr = thisConnManagerId + " Sent " + mb + " MB in " + ms + " ms at " + (mb / ms * @@ -92,11 +92,11 @@ private[spark] object ConnectionManagerTest extends Logging{ logInfo(resultStr) resultStr }).collect() - - println("---------------------") - println("Run " + i) + + println("---------------------") + println("Run " + i) resultStrs.foreach(println) - println("---------------------") + println("---------------------") }) } } diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 2b41c403b2e0a..9dc51e0d401f8 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -18,7 +18,7 @@ package org.apache.spark.network import java.nio.ByteBuffer -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} private[spark] object ReceiverTest { def main(args: Array[String]) { diff --git a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala b/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala index 0d9f743b3624b..a1dfc4094cca7 100644 --- a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala @@ -26,33 +26,33 @@ import org.apache.spark._ import org.apache.spark.network._ /** - * SecurityMessage is class that contains the connectionId and sasl token + * SecurityMessage is class that contains the connectionId and sasl token * used in SASL negotiation. SecurityMessage has routines for converting * it to and from a BufferMessage so that it can be sent by the ConnectionManager * and easily consumed by users when received. * The api was modeled after BlockMessage. * - * The connectionId is the connectionId of the client side. Since + * The connectionId is the connectionId of the client side. Since * message passing is asynchronous and its possible for the server side (receiving) - * to get multiple different types of messages on the same connection the connectionId - * is used to know which connnection the security message is intended for. - * + * to get multiple different types of messages on the same connection the connectionId + * is used to know which connnection the security message is intended for. + * * For instance, lets say we are node_0. We need to send data to node_1. The node_0 side * is acting as a client and connecting to node_1. SASL negotiation has to occur - * between node_0 and node_1 before node_1 trusts node_0 so node_0 sends a security message. - * node_1 receives the message from node_0 but before it can process it and send a response, - * some thread on node_1 decides it needs to send data to node_0 so it connects to node_0 - * and sends a security message of its own to authenticate as a client. Now node_0 gets - * the message and it needs to decide if this message is in response to it being a client - * (from the first send) or if its just node_1 trying to connect to it to send data. This + * between node_0 and node_1 before node_1 trusts node_0 so node_0 sends a security message. + * node_1 receives the message from node_0 but before it can process it and send a response, + * some thread on node_1 decides it needs to send data to node_0 so it connects to node_0 + * and sends a security message of its own to authenticate as a client. Now node_0 gets + * the message and it needs to decide if this message is in response to it being a client + * (from the first send) or if its just node_1 trying to connect to it to send data. This * is where the connectionId field is used. node_0 can lookup the connectionId to see if * it is in response to it being a client or if its in response to someone sending other data. - * + * * The format of a SecurityMessage as its sent is: * - Length of the ConnectionId - * - ConnectionId + * - ConnectionId * - Length of the token - * - Token + * - Token */ private[spark] class SecurityMessage() extends Logging { @@ -61,13 +61,13 @@ private[spark] class SecurityMessage() extends Logging { def set(byteArr: Array[Byte], newconnectionId: String) { if (byteArr == null) { - token = new Array[Byte](0) + token = new Array[Byte](0) } else { token = byteArr } connectionId = newconnectionId } - + /** * Read the given buffer and set the members of this class. */ @@ -91,17 +91,17 @@ private[spark] class SecurityMessage() extends Logging { buffer.clear() set(buffer) } - + def getConnectionId: String = { return connectionId } - + def getToken: Array[Byte] = { return token } - + /** - * Create a BufferMessage that can be sent by the ConnectionManager containing + * Create a BufferMessage that can be sent by the ConnectionManager containing * the security information from this class. * @return BufferMessage */ @@ -110,12 +110,12 @@ private[spark] class SecurityMessage() extends Logging { val buffers = new ArrayBuffer[ByteBuffer]() // 4 bytes for the length of the connectionId - // connectionId is of type char so multiple the length by 2 to get number of bytes + // connectionId is of type char so multiple the length by 2 to get number of bytes // 4 bytes for the length of token // token is a byte buffer so just take the length var buffer = ByteBuffer.allocate(4 + connectionId.length() * 2 + 4 + token.length) buffer.putInt(connectionId.length()) - connectionId.foreach((x: Char) => buffer.putChar(x)) + connectionId.foreach((x: Char) => buffer.putChar(x)) buffer.putInt(token.length) if (token.length > 0) { @@ -123,7 +123,7 @@ private[spark] class SecurityMessage() extends Logging { } buffer.flip() buffers += buffer - + var message = Message.createBufferMessage(buffers) logDebug("message total size is : " + message.size) message.isSecurityNeg = true @@ -136,7 +136,7 @@ private[spark] class SecurityMessage() extends Logging { } private[spark] object SecurityMessage { - + /** * Convert the given BufferMessage to a SecurityMessage by parsing the contents * of the BufferMessage and populating the SecurityMessage fields. diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala index 4164e81d3a8ae..136c1912045aa 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala @@ -36,8 +36,8 @@ private[spark] class FileHeader ( if (FileHeader.HEADER_SIZE - buf.readableBytes > 0 ) { buf.writeZero(FileHeader.HEADER_SIZE - buf.readableBytes) } else { - throw new Exception("too long header " + buf.readableBytes) - logInfo("too long header") + throw new Exception("too long header " + buf.readableBytes) + logInfo("too long header") } buf } diff --git a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala index eade07fbcbe37..cadd0c7ed19ba 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -44,7 +44,7 @@ class PartialResult[R](initialVal: R, isFinal: Boolean) { } } - /** + /** * Set a handler to be called when this PartialResult completes. Only one completion handler * is supported per PartialResult. */ @@ -60,7 +60,7 @@ class PartialResult[R](initialVal: R, isFinal: Boolean) { return this } - /** + /** * Set a handler to be called if this PartialResult's job fails. Only one failure handler * is supported per PartialResult. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 2306c9736b334..9ca971c8a4c27 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -52,7 +52,7 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { /** Compute the standard deviation of this RDD's elements. */ def stdev(): Double = stats().stdev - /** + /** * Compute the sample standard deviation of this RDD's elements (which corrects for bias in * estimating the standard deviation by dividing by N-1 instead of N). */ @@ -123,13 +123,13 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { * e.g. for the array * [1, 10, 20, 50] the buckets are [1, 10) [10, 20) [20, 50] * e.g 1<=x<10 , 10<=x<20, 20<=x<50 - * And on the input of 1 and 50 we would have a histogram of 1, 0, 0 - * + * And on the input of 1 and 50 we would have a histogram of 1, 0, 0 + * * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets * to true. * buckets must be sorted and not contain any duplicates. - * buckets array must be at least two elements + * buckets array must be at least two elements * All NaN entries are treated the same. If you have a NaN bucket it must be * the maximum value of the last position and all NaN entries will be counted * in that bucket. diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala index a84357b38414e..0c2cd7a24783b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -33,7 +33,7 @@ class PartitionerAwareUnionRDDPartition( val idx: Int ) extends Partition { var parents = rdds.map(_.partitions(idx)).toArray - + override val index = idx override def hashCode(): Int = idx diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 04c53d468465a..293cfb65643a6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -54,7 +54,7 @@ private[scheduler] case class BeginEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent private[scheduler] -case class GettingResultEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent +case class GettingResultEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent private[scheduler] case class CompletionEvent( task: Task[_], diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index 76f3e327d60b8..545fa453b7ccf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -1,107 +1,107 @@ -/* - * 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.scheduler - -import java.util.concurrent.LinkedBlockingQueue - -import org.apache.spark.Logging - -/** - * Asynchronously passes SparkListenerEvents to registered SparkListeners. - * - * Until start() is called, all posted events are only buffered. Only after this listener bus - * has started will events be actually propagated to all attached listeners. This listener bus - * is stopped when it receives a SparkListenerShutdown event, which is posted using stop(). - */ -private[spark] class LiveListenerBus extends SparkListenerBus with Logging { - - /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than - * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ - private val EVENT_QUEUE_CAPACITY = 10000 - private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) - private var queueFullErrorMessageLogged = false - private var started = false - private val listenerThread = new Thread("SparkListenerBus") { - setDaemon(true) - override def run() { - while (true) { - val event = eventQueue.take - if (event == SparkListenerShutdown) { - // Get out of the while loop and shutdown the daemon thread - return - } - postToAll(event) - } - } - } - - // Exposed for testing - @volatile private[spark] var stopCalled = false - - /** - * Start sending events to attached listeners. - * - * This first sends out all buffered events posted before this listener bus has started, then - * listens for any additional events asynchronously while the listener bus is still running. - * This should only be called once. - */ - def start() { - if (started) { - throw new IllegalStateException("Listener bus already started!") - } - listenerThread.start() - started = true - } - - def post(event: SparkListenerEvent) { - val eventAdded = eventQueue.offer(event) - if (!eventAdded && !queueFullErrorMessageLogged) { - logError("Dropping SparkListenerEvent because no remaining room in event queue. " + - "This likely means one of the SparkListeners is too slow and cannot keep up with the " + - "rate at which tasks are being started by the scheduler.") - queueFullErrorMessageLogged = true - } - } - - /** - * Waits until there are no more events in the queue, or until the specified time has elapsed. - * Used for testing only. Returns true if the queue has emptied and false is the specified time - * elapsed before the queue emptied. - */ - def waitUntilEmpty(timeoutMillis: Int): Boolean = { - val finishTime = System.currentTimeMillis + timeoutMillis - while (!eventQueue.isEmpty) { - if (System.currentTimeMillis > finishTime) { - return false - } - /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify - * add overhead in the general case. */ - Thread.sleep(10) - } - true - } - - def stop() { - stopCalled = true - if (!started) { - throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") - } - post(SparkListenerShutdown) - listenerThread.join() - } -} +/* + * 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.scheduler + +import java.util.concurrent.LinkedBlockingQueue + +import org.apache.spark.Logging + +/** + * Asynchronously passes SparkListenerEvents to registered SparkListeners. + * + * Until start() is called, all posted events are only buffered. Only after this listener bus + * has started will events be actually propagated to all attached listeners. This listener bus + * is stopped when it receives a SparkListenerShutdown event, which is posted using stop(). + */ +private[spark] class LiveListenerBus extends SparkListenerBus with Logging { + + /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than + * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ + private val EVENT_QUEUE_CAPACITY = 10000 + private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) + private var queueFullErrorMessageLogged = false + private var started = false + private val listenerThread = new Thread("SparkListenerBus") { + setDaemon(true) + override def run() { + while (true) { + val event = eventQueue.take + if (event == SparkListenerShutdown) { + // Get out of the while loop and shutdown the daemon thread + return + } + postToAll(event) + } + } + } + + // Exposed for testing + @volatile private[spark] var stopCalled = false + + /** + * Start sending events to attached listeners. + * + * This first sends out all buffered events posted before this listener bus has started, then + * listens for any additional events asynchronously while the listener bus is still running. + * This should only be called once. + */ + def start() { + if (started) { + throw new IllegalStateException("Listener bus already started!") + } + listenerThread.start() + started = true + } + + def post(event: SparkListenerEvent) { + val eventAdded = eventQueue.offer(event) + if (!eventAdded && !queueFullErrorMessageLogged) { + logError("Dropping SparkListenerEvent because no remaining room in event queue. " + + "This likely means one of the SparkListeners is too slow and cannot keep up with the " + + "rate at which tasks are being started by the scheduler.") + queueFullErrorMessageLogged = true + } + } + + /** + * Waits until there are no more events in the queue, or until the specified time has elapsed. + * Used for testing only. Returns true if the queue has emptied and false is the specified time + * elapsed before the queue emptied. + */ + def waitUntilEmpty(timeoutMillis: Int): Boolean = { + val finishTime = System.currentTimeMillis + timeoutMillis + while (!eventQueue.isEmpty) { + if (System.currentTimeMillis > finishTime) { + return false + } + /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify + * add overhead in the general case. */ + Thread.sleep(10) + } + true + } + + def stop() { + stopCalled = true + if (!started) { + throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") + } + post(SparkListenerShutdown) + listenerThread.join() + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 2fbbda5b76c74..ace9cd51c96b7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -240,7 +240,7 @@ object BlockFetcherIterator { override def numRemoteBlocks: Int = numRemote override def fetchWaitTime: Long = _fetchWaitTime override def remoteBytesRead: Long = _remoteBytesRead - + // Implementing the Iterator methods with an iterator that reads fetched blocks off the queue // as they arrive. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a2a729130091f..df9bb4044e37a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -388,7 +388,7 @@ private[spark] class BlockManager( logDebug("Block " + blockId + " not found in memory") } } - + // Look for the block in Tachyon if (level.useOffHeap) { logDebug("Getting block " + blockId + " from tachyon") @@ -1031,7 +1031,7 @@ private[spark] class BlockManager( memoryStore.clear() diskStore.clear() if (tachyonInitialized) { - tachyonStore.clear() + tachyonStore.clear() } metadataCleaner.cancel() broadcastCleaner.cancel() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala index 7168ae18c2615..337b45b727dec 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala @@ -37,7 +37,7 @@ private[spark] class BlockMessage() { private var id: BlockId = null private var data: ByteBuffer = null private var level: StorageLevel = null - + def set(getBlock: GetBlock) { typ = BlockMessage.TYPE_GET_BLOCK id = getBlock.id @@ -75,13 +75,13 @@ private[spark] class BlockMessage() { idBuilder += buffer.getChar() } id = BlockId(idBuilder.toString) - + if (typ == BlockMessage.TYPE_PUT_BLOCK) { val booleanInt = buffer.getInt() val replication = buffer.getInt() level = StorageLevel(booleanInt, replication) - + val dataLength = buffer.getInt() data = ByteBuffer.allocate(dataLength) if (dataLength != buffer.remaining) { @@ -108,12 +108,12 @@ private[spark] class BlockMessage() { buffer.clear() set(buffer) } - + def getType: Int = typ def getId: BlockId = id def getData: ByteBuffer = data def getLevel: StorageLevel = level - + def toBufferMessage: BufferMessage = { val startTime = System.currentTimeMillis val buffers = new ArrayBuffer[ByteBuffer]() @@ -127,7 +127,7 @@ private[spark] class BlockMessage() { buffer = ByteBuffer.allocate(8).putInt(level.toInt).putInt(level.replication) buffer.flip() buffers += buffer - + buffer = ByteBuffer.allocate(4).putInt(data.remaining) buffer.flip() buffers += buffer @@ -140,7 +140,7 @@ private[spark] class BlockMessage() { buffers += data } - + /* println() println("BlockMessage: ") @@ -158,7 +158,7 @@ private[spark] class BlockMessage() { } override def toString: String = { - "BlockMessage [type = " + typ + ", id = " + id + ", level = " + level + + "BlockMessage [type = " + typ + ", id = " + id + ", level = " + level + ", data = " + (if (data != null) data.remaining.toString else "null") + "]" } } @@ -168,7 +168,7 @@ private[spark] object BlockMessage { val TYPE_GET_BLOCK: Int = 1 val TYPE_GOT_BLOCK: Int = 2 val TYPE_PUT_BLOCK: Int = 3 - + def fromBufferMessage(bufferMessage: BufferMessage): BlockMessage = { val newBlockMessage = new BlockMessage() newBlockMessage.set(bufferMessage) @@ -192,7 +192,7 @@ private[spark] object BlockMessage { newBlockMessage.set(gotBlock) newBlockMessage } - + def fromPutBlock(putBlock: PutBlock): BlockMessage = { val newBlockMessage = new BlockMessage() newBlockMessage.set(putBlock) @@ -206,7 +206,7 @@ private[spark] object BlockMessage { val bMsg = B.toBufferMessage val C = new BlockMessage() C.set(bMsg) - + println(B.getId + " " + B.getLevel) println(C.getId + " " + C.getLevel) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala index dc62b1efaa7d4..973d85c0a9b3a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala @@ -27,16 +27,16 @@ import org.apache.spark.network._ private[spark] class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockMessage] with Logging { - + def this(bm: BlockMessage) = this(Array(bm)) def this() = this(null.asInstanceOf[Seq[BlockMessage]]) - def apply(i: Int) = blockMessages(i) + def apply(i: Int) = blockMessages(i) def iterator = blockMessages.iterator - def length = blockMessages.length + def length = blockMessages.length def set(bufferMessage: BufferMessage) { val startTime = System.currentTimeMillis @@ -62,15 +62,15 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) logDebug("Trying to convert buffer " + newBuffer + " to block message") val newBlockMessage = BlockMessage.fromByteBuffer(newBuffer) logDebug("Created " + newBlockMessage) - newBlockMessages += newBlockMessage + newBlockMessages += newBlockMessage buffer.position(buffer.position() + size) } val finishTime = System.currentTimeMillis logDebug("Converted block message array from buffer message in " + (finishTime - startTime) / 1000.0 + " s") - this.blockMessages = newBlockMessages + this.blockMessages = newBlockMessages } - + def toBufferMessage: BufferMessage = { val buffers = new ArrayBuffer[ByteBuffer]() @@ -83,7 +83,7 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) buffers ++= bufferMessage.buffers logDebug("Added " + bufferMessage) }) - + logDebug("Buffer list:") buffers.foreach((x: ByteBuffer) => logDebug("" + x)) /* @@ -103,13 +103,13 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) } private[spark] object BlockMessageArray { - + def fromBufferMessage(bufferMessage: BufferMessage): BlockMessageArray = { val newBlockMessageArray = new BlockMessageArray() newBlockMessageArray.set(bufferMessage) newBlockMessageArray } - + def main(args: Array[String]) { val blockMessages = (0 until 10).map { i => @@ -124,10 +124,10 @@ private[spark] object BlockMessageArray { } val blockMessageArray = new BlockMessageArray(blockMessages) println("Block message array created") - + val bufferMessage = blockMessageArray.toBufferMessage println("Converted to buffer message") - + val totalSize = bufferMessage.size val newBuffer = ByteBuffer.allocate(totalSize) newBuffer.clear() @@ -137,7 +137,7 @@ private[spark] object BlockMessageArray { buffer.rewind() }) newBuffer.flip - val newBufferMessage = Message.createBufferMessage(newBuffer) + val newBufferMessage = Message.createBufferMessage(newBuffer) println("Copied to new buffer message, size = " + newBufferMessage.size) val newBlockMessageArray = BlockMessageArray.fromBufferMessage(newBufferMessage) @@ -147,7 +147,7 @@ private[spark] object BlockMessageArray { case BlockMessage.TYPE_PUT_BLOCK => { val pB = PutBlock(blockMessage.getId, blockMessage.getData, blockMessage.getLevel) println(pB) - } + } case BlockMessage.TYPE_GET_BLOCK => { val gB = new GetBlock(blockMessage.getId) println(gB) 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 e1a1f209c9282..9ce0398d010a8 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -136,7 +136,7 @@ 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 => + case filter : String => if (!filter.isEmpty) { logInfo("Adding filter: " + filter) val holder : FilterHolder = new FilterHolder() @@ -151,7 +151,7 @@ private[spark] object JettyUtils extends Logging { if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) } } - val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, + val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) handlers.foreach { case(handler) => handler.addFilter(holder, "/*", enumDispatcher) } } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index a487924effbff..a7cf04b3cbb86 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -48,7 +48,7 @@ private[spark] object UIUtils { case _ =>
  • Storage
  • } val environment = page match { - case Environment => + case Environment =>
  • Environment
  • case _ =>
  • Environment
  • } diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index cdbbc65292188..2d05e09b10948 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -45,7 +45,7 @@ private[spark] object ClosureCleaner extends Logging { private def isClosure(cls: Class[_]): Boolean = { cls.getName.contains("$anonfun$") } - + // Get a list of the classes of the outer objects of a given closure object, obj; // the outer objects are defined as any closures that obj is nested within, plus // possibly the class that the outermost closure is in, if any. We stop searching @@ -63,7 +63,7 @@ private[spark] object ClosureCleaner extends Logging { } Nil } - + // Get a list of the outer objects for a given closure object. private def getOuterObjects(obj: AnyRef): List[AnyRef] = { for (f <- obj.getClass.getDeclaredFields if f.getName == "$outer") { @@ -76,7 +76,7 @@ private[spark] object ClosureCleaner extends Logging { } Nil } - + private def getInnerClasses(obj: AnyRef): List[Class[_]] = { val seen = Set[Class[_]](obj.getClass) var stack = List[Class[_]](obj.getClass) @@ -92,7 +92,7 @@ private[spark] object ClosureCleaner extends Logging { } return (seen - obj.getClass).toList } - + private def createNullValue(cls: Class[_]): AnyRef = { if (cls.isPrimitive) { new java.lang.Byte(0: Byte) // Should be convertible to any primitive type @@ -100,13 +100,13 @@ private[spark] object ClosureCleaner extends Logging { null } } - + def clean(func: AnyRef) { // TODO: cache outerClasses / innerClasses / accessedFields val outerClasses = getOuterClasses(func) val innerClasses = getInnerClasses(func) val outerObjects = getOuterObjects(func) - + val accessedFields = Map[Class[_], Set[String]]() for (cls <- outerClasses) accessedFields(cls) = Set[String]() @@ -143,7 +143,7 @@ private[spark] object ClosureCleaner extends Logging { field.set(outer, value) } } - + if (outer != null) { // logInfo("2: Setting $outer on " + func.getClass + " to " + outer); val field = func.getClass.getDeclaredField("$outer") @@ -151,7 +151,7 @@ private[spark] object ClosureCleaner extends Logging { field.set(func, outer) } } - + private def instantiateClass(cls: Class[_], outer: AnyRef, inInterpreter: Boolean): AnyRef = { // logInfo("Creating a " + cls + " with outer = " + outer) if (!inInterpreter) { @@ -192,7 +192,7 @@ class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor } } } - + override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) { // Check for calls a getter method for a variable in an interpreter wrapper object. @@ -209,12 +209,12 @@ class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM4) { var myName: String = null - + override def visit(version: Int, access: Int, name: String, sig: String, superName: String, interfaces: Array[String]) { myName = name } - + override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { new MethodVisitor(ASM4) { diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index d990fd49ef834..f2396f7c80a35 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -611,7 +611,7 @@ private[spark] object JsonProtocol { val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel) rddInfo.numCachedPartitions = numCachedPartitions rddInfo.memSize = memSize - rddInfo.tachyonSize = tachyonSize + rddInfo.tachyonSize = tachyonSize rddInfo.diskSize = diskSize rddInfo } diff --git a/core/src/main/scala/org/apache/spark/util/NextIterator.scala b/core/src/main/scala/org/apache/spark/util/NextIterator.scala index 8266e5e495efc..e5c732a5a559b 100644 --- a/core/src/main/scala/org/apache/spark/util/NextIterator.scala +++ b/core/src/main/scala/org/apache/spark/util/NextIterator.scala @@ -19,7 +19,7 @@ package org.apache.spark.util /** Provides a basic/boilerplate Iterator implementation. */ private[spark] abstract class NextIterator[U] extends Iterator[U] { - + private var gotNext = false private var nextValue: U = _ private var closed = false @@ -34,7 +34,7 @@ private[spark] abstract class NextIterator[U] extends Iterator[U] { * This convention is required because `null` may be a valid value, * and using `Option` seems like it might create unnecessary Some/None * instances, given some iterators might be called in a tight loop. - * + * * @return U, or set 'finished' when done */ protected def getNext(): U 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 732748a7ff82b..d80eed455c427 100644 --- a/core/src/main/scala/org/apache/spark/util/StatCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/StatCounter.scala @@ -62,10 +62,10 @@ 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) { + } else if (other.n != 0) { val delta = other.mu - mu if (other.n * 10 < n) { mu = mu + (delta * other.n) / (n + other.n) diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index 3c8f94a416c65..1a647fa1c9d84 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -136,7 +136,7 @@ object Vector { def ones(length: Int) = Vector(length, _ => 1) /** - * Creates this [[org.apache.spark.util.Vector]] of given length containing random numbers + * Creates this [[org.apache.spark.util.Vector]] of given length containing random numbers * between 0.0 and 1.0. Optional scala.util.Random number generator can be provided. */ def random(length: Int, random: Random = new XORShiftRandom()) = diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index 8a4cdea2fa7b1..7f220383f9f8b 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -25,28 +25,28 @@ import scala.util.hashing.MurmurHash3 import org.apache.spark.util.Utils.timeIt /** - * This class implements a XORShift random number generator algorithm + * This class implements a XORShift random number generator algorithm * Source: * Marsaglia, G. (2003). Xorshift RNGs. Journal of Statistical Software, Vol. 8, Issue 14. * @see Paper * This implementation is approximately 3.5 times faster than * {@link java.util.Random java.util.Random}, partly because of the algorithm, but also due - * to renouncing thread safety. JDK's implementation uses an AtomicLong seed, this class + * to renouncing thread safety. JDK's implementation uses an AtomicLong seed, this class * uses a regular Long. We can forgo thread safety since we use a new instance of the RNG * for each thread. */ private[spark] class XORShiftRandom(init: Long) extends JavaRandom(init) { - + def this() = this(System.nanoTime) private var seed = XORShiftRandom.hashSeed(init) // we need to just override next - this will be called by nextInt, nextDouble, // nextGaussian, nextLong, etc. - override protected def next(bits: Int): Int = { + override protected def next(bits: Int): Int = { var nextSeed = seed ^ (seed << 21) nextSeed ^= (nextSeed >>> 35) - nextSeed ^= (nextSeed << 4) + nextSeed ^= (nextSeed << 4) seed = nextSeed (nextSeed & ((1L << bits) -1)).asInstanceOf[Int] } @@ -89,7 +89,7 @@ private[spark] object XORShiftRandom { val million = 1e6.toInt val javaRand = new JavaRandom(seed) val xorRand = new XORShiftRandom(seed) - + // this is just to warm up the JIT - we're not timing anything timeIt(1e6.toInt) { javaRand.nextInt() @@ -97,9 +97,9 @@ private[spark] object XORShiftRandom { } val iters = timeIt(numIters)(_) - + /* Return results as a map instead of just printing to screen - in case the user wants to do something with them */ + in case the user wants to do something with them */ Map("javaTime" -> iters {javaRand.nextInt()}, "xorTime" -> iters {xorRand.nextInt()}) diff --git a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala index c5f24c66ce0c1..c645e4cbe8132 100644 --- a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala @@ -37,7 +37,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val securityManager = new SecurityManager(conf); val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = securityManager) System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) @@ -54,14 +54,14 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { assert(securityManagerBad.isAuthenticationEnabled() === true) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf, securityManager = securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) - intercept[akka.actor.ActorNotFound] { - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + intercept[akka.actor.ActorNotFound] { + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) } actorSystem.shutdown() @@ -75,7 +75,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val securityManager = new SecurityManager(conf); val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = securityManager) System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) @@ -91,7 +91,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { badconf.set("spark.authenticate.secret", "good") val securityManagerBad = new SecurityManager(badconf); - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = badconf, securityManager = securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( @@ -127,7 +127,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val securityManager = new SecurityManager(conf); val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = securityManager) System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) @@ -180,7 +180,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val securityManager = new SecurityManager(conf); val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = securityManager) System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) @@ -204,8 +204,8 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) - intercept[akka.actor.ActorNotFound] { - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + intercept[akka.actor.ActorNotFound] { + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) } actorSystem.shutdown() diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 9cbdfc54a3dc8..7f59bdcce4cc7 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -39,7 +39,7 @@ class DriverSuite extends FunSuite with Timeouts { failAfter(60 seconds) { Utils.executeAndGetOutput( Seq("./bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master), - new File(sparkHome), + new File(sparkHome), Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) } } diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index aee9ab9091dac..d651fbbac4e97 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -45,7 +45,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val pw = new PrintWriter(textFile) pw.println("100") pw.close() - + val jarFile = new File(tmpDir, "test.jar") val jarStream = new FileOutputStream(jarFile) val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest()) @@ -53,7 +53,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val jarEntry = new JarEntry(textFile.getName) jar.putNextEntry(jarEntry) - + val in = new FileInputStream(textFile) val buffer = new Array[Byte](10240) var nRead = 0 diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 01af94077144a..b9b668d3cc62a 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -106,7 +106,7 @@ class FileSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local", "test") val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath - val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), "a" * x)) + val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), "a" * x)) nums.saveAsSequenceFile(outputDir) // Try reading the output back as a SequenceFile val output = sc.sequenceFile[IntWritable, Text](outputDir) diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala index 0b5ed6d77034b..5e538d6fab2a1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala @@ -45,4 +45,4 @@ class WorkerWatcherSuite extends FunSuite { actorRef.underlyingActor.receive(new DisassociatedEvent(null, otherAkkaAddress, false)) assert(!actorRef.underlyingActor.isShutDown) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index 09e35bfc8f85f..e89b296d41026 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -42,7 +42,7 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { override def beforeAll() { sc = new SparkContext("local", "test") - + // Set the block size of local file system to test whether files are split right or not. sc.hadoopConfiguration.setLong("fs.local.block.size", 32) } diff --git a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala index a4381a8b974df..4df36558b6d4b 100644 --- a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala @@ -34,14 +34,14 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(1).mkString(",") === "2") assert(slices(2).mkString(",") === "3") } - + test("one slice") { val data = Array(1, 2, 3) val slices = ParallelCollectionRDD.slice(data, 1) assert(slices.size === 1) assert(slices(0).mkString(",") === "1,2,3") } - + test("equal slices") { val data = Array(1, 2, 3, 4, 5, 6, 7, 8, 9) val slices = ParallelCollectionRDD.slice(data, 3) @@ -50,7 +50,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(1).mkString(",") === "4,5,6") assert(slices(2).mkString(",") === "7,8,9") } - + test("non-equal slices") { val data = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) val slices = ParallelCollectionRDD.slice(data, 3) @@ -77,14 +77,14 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(1).mkString(",") === (33 to 66).mkString(",")) assert(slices(2).mkString(",") === (67 to 100).mkString(",")) } - + test("empty data") { val data = new Array[Int](0) val slices = ParallelCollectionRDD.slice(data, 5) assert(slices.size === 5) for (slice <- slices) assert(slice.size === 0) } - + test("zero slices") { val data = Array(1, 2, 3) intercept[IllegalArgumentException] { ParallelCollectionRDD.slice(data, 0) } @@ -94,7 +94,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = Array(1, 2, 3) intercept[IllegalArgumentException] { ParallelCollectionRDD.slice(data, -5) } } - + test("exclusive ranges sliced into ranges") { val data = 1 until 100 val slices = ParallelCollectionRDD.slice(data, 3) @@ -102,7 +102,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 99) assert(slices.forall(_.isInstanceOf[Range])) } - + test("inclusive ranges sliced into ranges") { val data = 1 to 100 val slices = ParallelCollectionRDD.slice(data, 3) @@ -124,7 +124,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(range.step === 1, "slice " + i + " step") } } - + test("random array tests") { val gen = for { d <- arbitrary[List[Int]] @@ -141,7 +141,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { } check(prop) } - + test("random exclusive range tests") { val gen = for { a <- Gen.choose(-100, 100) @@ -177,7 +177,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { } check(prop) } - + test("exclusive ranges of longs") { val data = 1L until 100L val slices = ParallelCollectionRDD.slice(data, 3) @@ -185,7 +185,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } - + test("inclusive ranges of longs") { val data = 1L to 100L val slices = ParallelCollectionRDD.slice(data, 3) @@ -193,7 +193,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 100) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } - + test("exclusive ranges of doubles") { val data = 1.0 until 100.0 by 1.0 val slices = ParallelCollectionRDD.slice(data, 3) @@ -201,7 +201,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } - + test("inclusive ranges of doubles") { val data = 1.0 to 100.0 by 1.0 val slices = ParallelCollectionRDD.slice(data, 3) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index dc704e07a81de..4cdccdda6f72e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -216,7 +216,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc test("onTaskGettingResult() called when result fetched remotely") { val listener = new SaveTaskEvents sc.addSparkListener(listener) - + // Make a task whose result is larger than the akka frame size System.setProperty("spark.akka.frameSize", "1") val akkaFrameSize = @@ -236,7 +236,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc test("onTaskGettingResult() not called when result sent directly") { val listener = new SaveTaskEvents sc.addSparkListener(listener) - + // Make a task whose result is larger than the akka frame size val result = sc.parallelize(Seq(1), 1).map(2 * _).reduce { case (x, y) => x } assert(result === 2) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 356e28dd19bc5..2fb750d9ee378 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -264,7 +264,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin test("Scheduler does not always schedule tasks on the same workers") { sc = new SparkContext("local", "TaskSchedulerImplSuite") - val taskScheduler = new TaskSchedulerImpl(sc) + val taskScheduler = new TaskSchedulerImpl(sc) taskScheduler.initialize(new FakeSchedulerBackend) // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. val dagScheduler = new DAGScheduler(sc, taskScheduler) { diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 45c322427930d..2f9739f940dc6 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -33,8 +33,8 @@ class UISuite extends FunSuite { val server = new Server(startPort) Try { server.start() } match { - case Success(s) => - case Failure(e) => + case Success(s) => + case Failure(e) => // Either case server port is busy hence setup for test complete } val serverInfo1 = JettyUtils.startJettyServer( diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index 439e5644e20a3..d7e48e633e0ee 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -69,7 +69,7 @@ object TestObject { class TestClass extends Serializable { var x = 5 - + def getX = x def run(): Int = { diff --git a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala index e1446cbc90bdb..32d74d0500b72 100644 --- a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala @@ -32,7 +32,7 @@ class NextIteratorSuite extends FunSuite with ShouldMatchers { i.hasNext should be === false intercept[NoSuchElementException] { i.next() } } - + test("two iterations") { val i = new StubIterator(Buffer(1, 2)) i.hasNext should be === true @@ -70,7 +70,7 @@ class NextIteratorSuite extends FunSuite with ShouldMatchers { class StubIterator(ints: Buffer[Int]) extends NextIterator[Int] { var closeCalled = 0 - + override def getNext() = { if (ints.size == 0) { finished = true diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index 757476efdb789..39199a1a17ccd 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -29,12 +29,12 @@ class XORShiftRandomSuite extends FunSuite with ShouldMatchers { val xorRand = new XORShiftRandom(seed) val hundMil = 1e8.toInt } - + /* - * This test is based on a chi-squared test for randomness. The values are hard-coded + * This test is based on a chi-squared test for randomness. The values are hard-coded * so as not to create Spark's dependency on apache.commons.math3 just to call one * method for calculating the exact p-value for a given number of random numbers - * and bins. In case one would want to move to a full-fledged test based on + * and bins. In case one would want to move to a full-fledged test based on * apache.commons.math3, the relevant class is here: * org.apache.commons.math3.stat.inference.ChiSquareTest */ @@ -49,19 +49,19 @@ class XORShiftRandomSuite extends FunSuite with ShouldMatchers { // populate bins based on modulus of the random number times(f.hundMil) {bins(math.abs(f.xorRand.nextInt) % 10) += 1} - /* since the seed is deterministic, until the algorithm is changed, we know the result will be - * exactly this: Array(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, - * 10000790, 10002286, 9998699), so the test will never fail at the prespecified (5%) - * significance level. However, should the RNG implementation change, the test should still - * pass at the same significance level. The chi-squared test done in R gave the following + /* since the seed is deterministic, until the algorithm is changed, we know the result will be + * exactly this: Array(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, + * 10000790, 10002286, 9998699), so the test will never fail at the prespecified (5%) + * significance level. However, should the RNG implementation change, the test should still + * pass at the same significance level. The chi-squared test done in R gave the following * results: * > chisq.test(c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, * 10000790, 10002286, 9998699)) * Chi-squared test for given probabilities - * data: c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, 10000790, + * data: c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, 10000790, * 10002286, 9998699) * X-squared = 11.975, df = 9, p-value = 0.2147 - * Note that the p-value was ~0.22. The test will fail if alpha < 0.05, which for 100 million + * Note that the p-value was ~0.22. The test will fail if alpha < 0.05, which for 100 million * random numbers * and 10 bins will happen at X-squared of ~16.9196. So, the test will fail if X-squared * is greater than or equal to that number. diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala index 41e813d48c7b8..1204cfba39f77 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala @@ -48,41 +48,41 @@ import org.apache.spark.streaming.dstream._ * @param storageLevel RDD storage level. */ -private[streaming] +private[streaming] class MQTTInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, brokerUrl: String, topic: String, storageLevel: StorageLevel ) extends NetworkInputDStream[T](ssc_) with Logging { - + def getReceiver(): NetworkReceiver[T] = { new MQTTReceiver(brokerUrl, topic, storageLevel).asInstanceOf[NetworkReceiver[T]] } } -private[streaming] +private[streaming] class MQTTReceiver(brokerUrl: String, topic: String, storageLevel: StorageLevel ) extends NetworkReceiver[Any] { lazy protected val blockGenerator = new BlockGenerator(storageLevel) - + def onStop() { blockGenerator.stop() } - + def onStart() { blockGenerator.start() - // Set up persistence for messages + // Set up persistence for messages var peristance: MqttClientPersistence = new MemoryPersistence() // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance var client: MqttClient = new MqttClient(brokerUrl, MqttClient.generateClientId(), peristance) - // Connect to MqttBroker + // Connect to MqttBroker client.connect() // Subscribe to Mqtt topic @@ -91,7 +91,7 @@ class MQTTReceiver(brokerUrl: String, // Callback automatically triggers as and when new message arrives on specified topic var callback: MqttCallback = new MqttCallback() { - // Handles Mqtt message + // Handles Mqtt message override def messageArrived(arg0: String, arg1: MqttMessage) { blockGenerator += new String(arg1.getPayload()) } diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala index 3316b6dc39d6b..843a4a7a9ad72 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala @@ -31,7 +31,7 @@ import org.apache.spark.storage.StorageLevel * @constructor create a new Twitter stream using the supplied Twitter4J authentication credentials. * An optional set of string filters can be used to restrict the set of tweets. The Twitter API is * such that this may return a sampled subset of all tweets during each interval. -* +* * If no Authorization object is provided, initializes OAuth authorization using the system * properties twitter4j.oauth.consumerKey, .consumerSecret, .accessToken and .accessTokenSecret. */ @@ -42,13 +42,13 @@ class TwitterInputDStream( filters: Seq[String], storageLevel: StorageLevel ) extends NetworkInputDStream[Status](ssc_) { - + private def createOAuthAuthorization(): Authorization = { new OAuthAuthorization(new ConfigurationBuilder().build()) } private val authorization = twitterAuth.getOrElse(createOAuthAuthorization()) - + override def getReceiver(): NetworkReceiver[Status] = { new TwitterReceiver(authorization, filters, storageLevel) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 377d9d6bd5e72..5635287694ee2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -172,7 +172,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali "EdgeDirection.Either instead.") } } - + /** * Join the vertices with an RDD and then apply a function from the * the vertex and RDD entry to a new vertex value. The input table diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala index 6386306c048fc..a467ca1ae715a 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -55,7 +55,7 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { } } } - + test ("filter") { withSpark { sc => val n = 5 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index e41d9bbe18c37..7f6d94571b5ef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -30,7 +30,7 @@ import org.apache.spark.mllib.linalg.Vector trait Optimizer extends Serializable { /** - * Solve the provided convex optimization problem. + * Solve the provided convex optimization problem. */ def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 3bd0017aa196a..d969e7aa60061 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -26,7 +26,7 @@ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg.{Vectors, Vector} /** - * GeneralizedLinearModel (GLM) represents a model trained using + * GeneralizedLinearModel (GLM) represents a model trained using * GeneralizedLinearAlgorithm. GLMs consist of a weight vector and * an intercept. * @@ -38,7 +38,7 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double /** * Predict the result given a data point and the weights learned. - * + * * @param dataMatrix Row vector containing the features for this data point * @param weightMatrix Column vector containing the weights of the model * @param intercept Intercept of the model. diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index a30dcfdcecf27..687e85ca94d3c 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -35,7 +35,7 @@ import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ * A ClassLoader that reads classes from a Hadoop FileSystem or HTTP URI, * used to load classes defined by the interpreter when the REPL is used. * Allows the user to specify if user class path should be first - */ + */ class ExecutorClassLoader(classUri: String, parent: ClassLoader, userClassPathFirst: Boolean) extends ClassLoader { val uri = new URI(classUri) @@ -94,7 +94,7 @@ class ExecutorClassLoader(classUri: String, parent: ClassLoader, case e: Exception => None } } - + def readAndTransformClass(name: String, in: InputStream): Array[Byte] = { if (name.startsWith("line") && name.endsWith("$iw$")) { // Class seems to be an interpreter "wrapper" object storing a val or var. diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala index 8f61a5e835044..419796b68b113 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala @@ -187,7 +187,7 @@ trait SparkImports { if (currentImps contains imv) addWrapper() val objName = req.lineRep.readPath val valName = "$VAL" + newValId(); - + if(!code.toString.endsWith(".`" + imv + "`;\n")) { // Which means already imported code.append("val " + valName + " = " + objName + ".INSTANCE;\n") code.append("import " + valName + req.accessPath + ".`" + imv + "`;\n") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 17118499d0c87..1f3fab09e9566 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -28,7 +28,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { override def toString = s"CAST($child, $dataType)" type EvaluatedType = Any - + def nullOrCast[T](a: Any, func: T => Any): Any = if(a == null) { null } else { @@ -40,7 +40,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case BinaryType => nullOrCast[Array[Byte]](_, new String(_, "UTF-8")) case _ => nullOrCast[Any](_, _.toString) } - + // BinaryConverter def castToBinary: Any => Any = child.dataType match { case StringType => nullOrCast[String](_, _.getBytes("UTF-8")) @@ -58,7 +58,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case DoubleType => nullOrCast[Double](_, _ != 0) case FloatType => nullOrCast[Float](_, _ != 0) } - + // TimestampConverter def castToTimestamp: Any => Any = child.dataType match { case StringType => nullOrCast[String](_, s => { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 8a1db8e796816..dd9332ada80dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -86,7 +86,7 @@ abstract class Expression extends TreeNode[Expression] { } /** - * Evaluation helper function for 2 Numeric children expressions. Those expressions are supposed + * Evaluation helper function for 2 Numeric children expressions. Those expressions are supposed * to be in the same data type, and also the return type. * Either one of the expressions result is null, the evaluation result should be null. */ @@ -120,7 +120,7 @@ abstract class Expression extends TreeNode[Expression] { } /** - * Evaluation helper function for 2 Fractional children expressions. Those expressions are + * Evaluation helper function for 2 Fractional children expressions. Those expressions are * supposed to be in the same data type, and also the return type. * Either one of the expressions result is null, the evaluation result should be null. */ @@ -153,7 +153,7 @@ abstract class Expression extends TreeNode[Expression] { } /** - * Evaluation helper function for 2 Integral children expressions. Those expressions are + * Evaluation helper function for 2 Integral children expressions. Those expressions are * supposed to be in the same data type, and also the return type. * Either one of the expressions result is null, the evaluation result should be null. */ @@ -186,12 +186,12 @@ abstract class Expression extends TreeNode[Expression] { } /** - * Evaluation helper function for 2 Comparable children expressions. Those expressions are + * Evaluation helper function for 2 Comparable children expressions. Those expressions are * supposed to be in the same data type, and the return type should be Integer: * Negative value: 1st argument less than 2nd argument * Zero: 1st argument equals 2nd argument * Positive value: 1st argument greater than 2nd argument - * + * * Either one of the expressions result is null, the evaluation result should be null. */ @inline @@ -213,7 +213,7 @@ abstract class Expression extends TreeNode[Expression] { null } else { e1.dataType match { - case i: NativeType => + case i: NativeType => f.asInstanceOf[(Ordering[i.JvmType], i.JvmType, i.JvmType) => Boolean]( i.ordering, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) case other => sys.error(s"Type $other does not support ordered operations") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index a27c71db1b999..ddc16ce87b895 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -28,19 +28,19 @@ trait StringRegexExpression { self: BinaryExpression => type EvaluatedType = Any - + def escape(v: String): String def matches(regex: Pattern, str: String): Boolean - + def nullable: Boolean = true def dataType: DataType = BooleanType - - // try cache the pattern for Literal + + // try cache the pattern for Literal private lazy val cache: Pattern = right match { case x @ Literal(value: String, StringType) => compile(value) case _ => null } - + protected def compile(str: String): Pattern = if(str == null) { null } else { @@ -49,7 +49,7 @@ trait StringRegexExpression { } protected def pattern(str: String) = if(cache == null) compile(str) else cache - + override def eval(input: Row): Any = { val l = left.eval(input) if (l == null) { @@ -73,11 +73,11 @@ trait StringRegexExpression { /** * Simple RegEx pattern matching function */ -case class Like(left: Expression, right: Expression) +case class Like(left: Expression, right: Expression) extends BinaryExpression with StringRegexExpression { - + def symbol = "LIKE" - + // replace the _ with .{1} exactly match 1 time of any character // replace the % with .*, match 0 or more times with any character override def escape(v: String) = { @@ -98,19 +98,19 @@ case class Like(left: Expression, right: Expression) sb.append(Pattern.quote(Character.toString(n))); } } - + i += 1 } - + sb.toString() } - + override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).matches() } -case class RLike(left: Expression, right: Expression) +case class RLike(left: Expression, right: Expression) extends BinaryExpression with StringRegexExpression { - + def symbol = "RLIKE" override def escape(v: String): String = v override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index cdeb01a9656f4..da34bd3a21503 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -55,9 +55,9 @@ case object BooleanType extends NativeType { case object TimestampType extends NativeType { type JvmType = Timestamp - + @transient lazy val tag = typeTag[JvmType] - + val ordering = new Ordering[JvmType] { def compare(x: Timestamp, y: Timestamp) = x.compareTo(y) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 888a19d79f7e4..2cd0d2b0e1385 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -144,7 +144,7 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("abc" like "b%", false) checkEvaluation("abc" like "bc%", false) } - + test("LIKE Non-literal Regular Expression") { val regEx = 'a.string.at(0) checkEvaluation("abcd" like regEx, null, new GenericRow(Array[Any](null))) @@ -164,7 +164,7 @@ class ExpressionEvaluationSuite extends FunSuite { test("RLIKE literal Regular Expression") { checkEvaluation("abdef" rlike "abdef", true) checkEvaluation("abbbbc" rlike "a.*c", true) - + checkEvaluation("fofo" rlike "^fo", true) checkEvaluation("fo\no" rlike "^fo\no$", true) checkEvaluation("Bn" rlike "^Ba*n", true) @@ -196,9 +196,9 @@ class ExpressionEvaluationSuite extends FunSuite { evaluate("abbbbc" rlike regEx, new GenericRow(Array[Any]("**"))) } } - + test("data type casting") { - + val sts = "1970-01-01 00:00:01.0" val ts = Timestamp.valueOf(sts) @@ -236,7 +236,7 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("23" cast ShortType, 23) checkEvaluation("2012-12-11" cast DoubleType, null) checkEvaluation(Literal(123) cast IntegerType, 123) - + intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)} } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 65eae3357a21e..1cbf973c34917 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -56,4 +56,4 @@ class ScalaReflectionRelationSuite extends FunSuite { val result = sql("SELECT data FROM reflectBinary").collect().head(0).asInstanceOf[Array[Byte]] assert(result.toSeq === Seq[Byte](1)) } -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 93023e8dced57..ac56ff709c1c4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -59,7 +59,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) } } -private[streaming] +private[streaming] object Checkpoint extends Logging { val PREFIX = "checkpoint-" val REGEX = (PREFIX + """([\d]+)([\w\.]*)""").r @@ -79,7 +79,7 @@ object Checkpoint extends Logging { def sortFunc(path1: Path, path2: Path): Boolean = { val (time1, bk1) = path1.getName match { case REGEX(x, y) => (x.toLong, !y.isEmpty) } val (time2, bk2) = path2.getName match { case REGEX(x, y) => (x.toLong, !y.isEmpty) } - (time1 < time2) || (time1 == time2 && bk1) + (time1 < time2) || (time1 == time2 && bk1) } val path = new Path(checkpointDir) @@ -95,7 +95,7 @@ object Checkpoint extends Logging { } } else { logInfo("Checkpoint directory " + path + " does not exist") - Seq.empty + Seq.empty } } } @@ -160,7 +160,7 @@ class CheckpointWriter( }) } - // All done, print success + // All done, print success val finishTime = System.currentTimeMillis() logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + checkpointFile + "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " ms") @@ -227,14 +227,14 @@ object CheckpointReader extends Logging { { val checkpointPath = new Path(checkpointDir) def fs = checkpointPath.getFileSystem(hadoopConf) - - // Try to find the checkpoint files + + // Try to find the checkpoint files val checkpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, fs).reverse if (checkpointFiles.isEmpty) { return None } - // Try to read the checkpoint files in the order + // Try to read the checkpoint files in the order logInfo("Checkpoint files found: " + checkpointFiles.mkString(",")) val compressionCodec = CompressionCodec.createCodec(conf) checkpointFiles.foreach(file => { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala index 16479a01272aa..ad4f3fdd14ad6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala @@ -20,11 +20,11 @@ package org.apache.spark.streaming private[streaming] class Interval(val beginTime: Time, val endTime: Time) { def this(beginMs: Long, endMs: Long) = this(new Time(beginMs), new Time(endMs)) - + def duration(): Duration = endTime - beginTime def + (time: Duration): Interval = { - new Interval(beginTime + time, endTime + time) + new Interval(beginTime + time, endTime + time) } def - (time: Duration): Interval = { @@ -40,9 +40,9 @@ class Interval(val beginTime: Time, val endTime: Time) { } def <= (that: Interval) = (this < that || this == that) - + def > (that: Interval) = !(this <= that) - + def >= (that: Interval) = !(this < that) override def toString = "[" + beginTime + ", " + endTime + "]" diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala index 2678334f53844..6a6b00a778b48 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala @@ -32,7 +32,7 @@ case class Time(private val millis: Long) { def <= (that: Time): Boolean = (this.millis <= that.millis) def > (that: Time): Boolean = (this.millis > that.millis) - + def >= (that: Time): Boolean = (this.millis >= that.millis) def + (that: Duration): Time = new Time(millis + that.milliseconds) @@ -43,7 +43,7 @@ case class Time(private val millis: Long) { def floor(that: Duration): Time = { val t = that.milliseconds - val m = math.floor(this.millis / t).toLong + val m = math.floor(this.millis / t).toLong new Time(m * t) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala index 903e3f3c9b713..f33c0ceafdf42 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala @@ -51,7 +51,7 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) .map(x => (x._1, x._2.getCheckpointFile.get)) logDebug("Current checkpoint files:\n" + checkpointFiles.toSeq.mkString("\n")) - // Add the checkpoint files to the data to be serialized + // Add the checkpoint files to the data to be serialized if (!checkpointFiles.isEmpty) { currentCheckpointFiles.clear() currentCheckpointFiles ++= checkpointFiles diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 8a6051622e2d5..e878285f6a854 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -232,7 +232,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas } logDebug("Accepted " + path) } catch { - case fnfe: java.io.FileNotFoundException => + case fnfe: java.io.FileNotFoundException => logWarning("Error finding new files", fnfe) reset() return false diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala index 97325f8ea3117..6376cff78b78a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala @@ -31,11 +31,11 @@ class QueueInputDStream[T: ClassTag]( oneAtATime: Boolean, defaultRDD: RDD[T] ) extends InputDStream[T](ssc) { - + override def start() { } - + override def stop() { } - + override def compute(validTime: Time): Option[RDD[T]] = { val buffer = new ArrayBuffer[RDD[T]]() if (oneAtATime && queue.size > 0) { @@ -55,5 +55,5 @@ class QueueInputDStream[T: ClassTag]( None } } - + } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala index 44eb2750c6c7a..f5984d03c5342 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala @@ -47,7 +47,7 @@ object ReceiverSupervisorStrategy { * the API for pushing received data into Spark Streaming for being processed. * * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html - * + * * @example {{{ * class MyActor extends Actor with Receiver{ * def receive { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala index c5ef2cc8c390d..39145a3ab081a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala @@ -19,34 +19,34 @@ package org.apache.spark.streaming.util private[streaming] trait Clock { - def currentTime(): Long + def currentTime(): Long def waitTillTime(targetTime: Long): Long } private[streaming] class SystemClock() extends Clock { - + val minPollTime = 25L - + def currentTime(): Long = { System.currentTimeMillis() - } - + } + def waitTillTime(targetTime: Long): Long = { var currentTime = 0L currentTime = System.currentTimeMillis() - + var waitTime = targetTime - currentTime if (waitTime <= 0) { return currentTime } - + val pollTime = { if (waitTime / 10.0 > minPollTime) { (waitTime / 10.0).toLong } else { - minPollTime - } + minPollTime + } } while (true) { @@ -55,7 +55,7 @@ class SystemClock() extends Clock { if (waitTime <= 0) { return currentTime } - val sleepTime = + val sleepTime = if (waitTime < pollTime) { waitTime } else { @@ -69,7 +69,7 @@ class SystemClock() extends Clock { private[streaming] class ManualClock() extends Clock { - + var time = 0L def currentTime() = time @@ -85,13 +85,13 @@ class ManualClock() extends Clock { this.synchronized { time += timeToAdd this.notifyAll() - } + } } def waitTillTime(targetTime: Long): Long = { this.synchronized { while (time < targetTime) { this.wait(100) - } + } } currentTime() } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala index 07021ebb5802a..bd1df55cf70f5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala @@ -25,8 +25,8 @@ import scala.collection.JavaConversions.mapAsScalaMap private[streaming] object RawTextHelper { - /** - * Splits lines and counts the words in them using specialized object-to-long hashmap + /** + * Splits lines and counts the words in them using specialized object-to-long hashmap * (to avoid boxing-unboxing overhead of Long in java/scala HashMap) */ def splitAndCountPartitions(iter: Iterator[String]): Iterator[(String, Long)] = { @@ -55,13 +55,13 @@ object RawTextHelper { map.toIterator.map{case (k, v) => (k, v)} } - /** + /** * Gets the top k words in terms of word counts. Assumes that each word exists only once * in the `data` iterator (that is, the counts have been reduced). */ def topK(data: Iterator[(String, Long)], k: Int): Iterator[(String, Long)] = { val taken = new Array[(String, Long)](k) - + var i = 0 var len = 0 var done = false @@ -93,7 +93,7 @@ object RawTextHelper { } taken.toIterator } - + /** * Warms up the SparkContext in master and slave by running tasks to force JIT kick in * before real workload starts. @@ -106,11 +106,11 @@ object RawTextHelper { .count() } } - - def add(v1: Long, v2: Long) = (v1 + v2) - def subtract(v1: Long, v2: Long) = (v1 - v2) + def add(v1: Long, v2: Long) = (v1 + v2) + + def subtract(v1: Long, v2: Long) = (v1 - v2) - def max(v1: Long, v2: Long) = math.max(v1, v2) + def max(v1: Long, v2: Long) = math.max(v1, v2) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala index f71938ac55ccb..e016377c94c0d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala @@ -22,10 +22,10 @@ import org.apache.spark.Logging private[streaming] class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name: String) extends Logging { - + private val thread = new Thread("RecurringTimer - " + name) { setDaemon(true) - override def run() { loop } + override def run() { loop } } @volatile private var prevTime = -1L @@ -104,11 +104,11 @@ class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name: private[streaming] object RecurringTimer { - + def main(args: Array[String]) { var lastRecurTime = 0L val period = 1000 - + def onRecur(time: Long) { val currentTime = System.currentTimeMillis() println("" + currentTime + ": " + (currentTime - lastRecurTime)) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 13fa64894b773..a0b1bbc34fa7c 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -1673,7 +1673,7 @@ public void testSocketTextStream() { @Test public void testSocketString() { - + class Converter implements Function> { public Iterable call(InputStream in) throws IOException { BufferedReader reader = new BufferedReader(new InputStreamReader(in)); From f99401a6308d5b9a9259d7597a35ba92f927aa50 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 10 Apr 2014 16:20:33 -0700 Subject: [PATCH 049/641] [SQL] Improve column pruning in the optimizer. Author: Michael Armbrust Closes #378 from marmbrus/columnPruning and squashes the following commits: 779da56 [Michael Armbrust] More consistent naming. 1a4e9ea [Michael Armbrust] More comments. 2f4e7b9 [Michael Armbrust] Improve column pruning in the optimizer. --- .../sql/catalyst/optimizer/Optimizer.scala | 51 ++++++++++++++++++- .../plans/logical/basicOperators.scala | 2 +- 2 files changed, 51 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 37b23ba58289c..c0a09a16ac98d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -33,7 +33,56 @@ object Optimizer extends RuleExecutor[LogicalPlan] { Batch("Filter Pushdown", Once, CombineFilters, PushPredicateThroughProject, - PushPredicateThroughInnerJoin) :: Nil + PushPredicateThroughInnerJoin, + ColumnPruning) :: Nil +} + +/** + * Attempts to eliminate the reading of unneeded columns from the query plan using the following + * transformations: + * + * - Inserting Projections beneath the following operators: + * - Aggregate + * - Project <- Join + * - Collapse adjacent projections, performing alias substitution. + */ +object ColumnPruning extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case a @ Aggregate(_, _, child) if (child.outputSet -- a.references).nonEmpty => + // Project away references that are not needed to calculate the required aggregates. + a.copy(child = Project(a.references.toSeq, child)) + + case Project(projectList, Join(left, right, joinType, condition)) => + // Collect the list of off references required either above or to evaluate the condition. + val allReferences: Set[Attribute] = + projectList.flatMap(_.references).toSet ++ condition.map(_.references).getOrElse(Set.empty) + /** Applies a projection when the child is producing unnecessary attributes */ + def prunedChild(c: LogicalPlan) = + if ((allReferences.filter(c.outputSet.contains) -- c.outputSet).nonEmpty) { + Project(allReferences.filter(c.outputSet.contains).toSeq, c) + } else { + c + } + + Project(projectList, Join(prunedChild(left), prunedChild(right), joinType, condition)) + + case Project(projectList1, Project(projectList2, child)) => + // Create a map of Aliases to their values from the child projection. + // e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c -> Alias(a + b, c)). + val aliasMap = projectList2.collect { + case a @ Alias(e, _) => (a.toAttribute: Expression, a) + }.toMap + + // Substitute any attributes that are produced by the child projection, so that we safely + // eliminate it. + // e.g., 'SELECT c + 1 FROM (SELECT a + b AS C ...' produces 'SELECT a + b + 1 ...' + // TODO: Fix TransformBase to avoid the cast below. + val substitutedProjection = projectList1.map(_.transform { + case a if aliasMap.contains(a) => aliasMap(a) + }).asInstanceOf[Seq[NamedExpression]] + + Project(substitutedProjection, child) + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index cfc0b0c3a8d98..397473e178867 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -127,7 +127,7 @@ case class Aggregate( extends UnaryNode { def output = aggregateExpressions.map(_.toAttribute) - def references = child.references + def references = (groupingExpressions ++ aggregateExpressions).flatMap(_.references).toSet } case class Limit(limit: Expression, child: LogicalPlan) extends UnaryNode { From 2c557837b4a12c644cc37bd00d02be04f3807637 Mon Sep 17 00:00:00 2001 From: Sundeep Narravula Date: Thu, 10 Apr 2014 17:10:11 -0700 Subject: [PATCH 050/641] SPARK-1202 - Add a "cancel" button in the UI for stages Author: Sundeep Narravula Author: Sundeep Narravula Closes #246 from sundeepn/uikilljob and squashes the following commits: 5fdd0e2 [Sundeep Narravula] Fix test string f6fdff1 [Sundeep Narravula] Format fix; reduced line size to less than 100 chars d1daeb9 [Sundeep Narravula] Incorporating review comments. 8d97923 [Sundeep Narravula] Ability to kill jobs thru the UI. This behavior can be turned on be settings the following variable: spark.ui.killEnabled=true (default=false) Adding DAGScheduler event StageCancelled and corresponding handlers. Added cancellation reason to handlers. --- .../scala/org/apache/spark/SparkContext.scala | 10 ++++++ .../apache/spark/scheduler/DAGScheduler.scala | 32 ++++++++++++++++--- .../spark/scheduler/DAGSchedulerEvent.scala | 2 ++ .../scala/org/apache/spark/ui/SparkUI.scala | 1 + .../org/apache/spark/ui/jobs/IndexPage.scala | 14 +++++++- .../apache/spark/ui/jobs/JobProgressUI.scala | 1 + .../org/apache/spark/ui/jobs/StagePage.scala | 1 + .../org/apache/spark/ui/jobs/StageTable.scala | 29 +++++++++++++---- .../spark/scheduler/DAGSchedulerSuite.scala | 2 +- docs/configuration.md | 7 ++++ 10 files changed, 87 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e6c9b7000d819..3bcc8ce2b25a6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1138,6 +1138,16 @@ class SparkContext(config: SparkConf) extends Logging { dagScheduler.cancelAllJobs() } + /** Cancel a given job if it's scheduled or running */ + private[spark] def cancelJob(jobId: Int) { + dagScheduler.cancelJob(jobId) + } + + /** Cancel a given stage and all jobs associated with it */ + private[spark] def cancelStage(stageId: Int) { + dagScheduler.cancelStage(stageId) + } + /** * Clean a closure to make it ready to serialized and send to tasks * (removes unreferenced variables in $outer's, updates REPL variables) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c41d6d75a1d49..c6cbf14e20069 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -511,6 +511,13 @@ class DAGScheduler( eventProcessActor ! AllJobsCancelled } + /** + * Cancel all jobs associated with a running or scheduled stage. + */ + def cancelStage(stageId: Int) { + eventProcessActor ! StageCancelled(stageId) + } + /** * Process one event retrieved from the event processing actor. * @@ -551,6 +558,9 @@ class DAGScheduler( submitStage(finalStage) } + case StageCancelled(stageId) => + handleStageCancellation(stageId) + case JobCancelled(jobId) => handleJobCancellation(jobId) @@ -560,11 +570,13 @@ class DAGScheduler( val activeInGroup = activeJobs.filter(activeJob => groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) val jobIds = activeInGroup.map(_.jobId) - jobIds.foreach(handleJobCancellation) + jobIds.foreach(jobId => handleJobCancellation(jobId, + "as part of cancelled job group %s".format(groupId))) case AllJobsCancelled => // Cancel all running jobs. - runningStages.map(_.jobId).foreach(handleJobCancellation) + runningStages.map(_.jobId).foreach(jobId => handleJobCancellation(jobId, + "as part of cancellation of all jobs")) activeJobs.clear() // These should already be empty by this point, jobIdToActiveJob.clear() // but just in case we lost track of some jobs... @@ -991,11 +1003,23 @@ class DAGScheduler( } } - private def handleJobCancellation(jobId: Int) { + private def handleStageCancellation(stageId: Int) { + if (stageIdToJobIds.contains(stageId)) { + val jobsThatUseStage: Array[Int] = stageIdToJobIds(stageId).toArray + jobsThatUseStage.foreach(jobId => { + handleJobCancellation(jobId, "because Stage %s was cancelled".format(stageId)) + }) + } else { + logInfo("No active jobs to kill for Stage " + stageId) + } + } + + private def handleJobCancellation(jobId: Int, reason: String = "") { if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { - failJobAndIndependentStages(jobIdToActiveJob(jobId), s"Job $jobId cancelled", None) + failJobAndIndependentStages(jobIdToActiveJob(jobId), + "Job %d cancelled %s".format(jobId, reason), None) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 293cfb65643a6..7367c08b5d324 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -44,6 +44,8 @@ private[scheduler] case class JobSubmitted( properties: Properties = null) extends DAGSchedulerEvent +private[scheduler] case class StageCancelled(stageId: Int) extends DAGSchedulerEvent + private[scheduler] case class JobCancelled(jobId: Int) extends DAGSchedulerEvent private[scheduler] case class JobGroupCancelled(groupId: String) extends DAGSchedulerEvent diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index b8e6e15880bf5..dac11ec1cf52f 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -46,6 +46,7 @@ private[spark] class SparkUI( val live = sc != null val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) + val killEnabled = conf.getBoolean("spark.ui.killEnabled", true) private val localHost = Utils.localHostName() private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index f811aff616bcf..5da5d1f2a3f45 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -32,6 +32,7 @@ private[ui] class IndexPage(parent: JobProgressUI) { private val sc = parent.sc private lazy val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler + private val killEnabled = parent.killEnabled private def appName = parent.appName @@ -42,7 +43,18 @@ private[ui] class IndexPage(parent: JobProgressUI) { val failedStages = listener.failedStages.reverse.toSeq val now = System.currentTimeMillis() - val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) + if (killEnabled) { + val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean + val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt + + if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) { + sc.cancelStage(stageId) + } + } + + + val activeStagesTable = + new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent, parent.killEnabled) val completedStagesTable = new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index ad1a12cdc4e36..9de659d6c7393 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -32,6 +32,7 @@ private[ui] class JobProgressUI(parent: SparkUI) { val basePath = parent.basePath val live = parent.live val sc = parent.sc + val killEnabled = parent.killEnabled lazy val listener = _listener.get lazy val isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 0bcbd7461cc5b..b6c3e3cf45163 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -30,6 +30,7 @@ import org.apache.spark.util.{Utils, Distribution} private[ui] class StagePage(parent: JobProgressUI) { private val basePath = parent.basePath private lazy val listener = parent.listener + private lazy val sc = parent.sc private def appName = parent.appName diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index ac61568af52d2..1e874ae4969f9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -27,7 +27,11 @@ import org.apache.spark.ui.{WebUI, UIUtils} import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ -private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { +private[ui] class StageTable( + stages: Seq[StageInfo], + parent: JobProgressUI, + killEnabled: Boolean = false) { + private val basePath = parent.basePath private lazy val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler @@ -71,15 +75,28 @@ private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { } - /** Render an HTML row that represents a stage */ - private def stageRow(s: StageInfo): Seq[Node] = { - val poolName = listener.stageIdToPool.get(s.stageId) + private def makeDescription(s: StageInfo): Seq[Node] = { val nameLink = {s.name} + val killLink = if (killEnabled) { + + + } val description = listener.stageIdToDescription.get(s.stageId) - .map(d =>
    {d}
    {nameLink}
    ).getOrElse(nameLink) + .map(d =>
    {d}
    {nameLink} {killLink}
    ) + .getOrElse(
    {nameLink} {killLink}
    ) + + return description + } + + /** Render an HTML row that represents a stage */ + private def stageRow(s: StageInfo): Seq[Node] = { + val poolName = listener.stageIdToPool.get(s.stageId) val submissionTime = s.submissionTime match { case Some(t) => WebUI.formatDate(new Date(t)) case None => "Unknown" @@ -118,7 +135,7 @@ private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { }} - {description} + {makeDescription(s)} {submissionTime} {formattedDuration} diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index a74724d785ad3..db4df1d1212ff 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -290,7 +290,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val rdd = makeRdd(1, Nil) val jobId = submit(rdd, Array(0)) cancel(jobId) - assert(failure.getMessage === s"Job $jobId cancelled") + assert(failure.getMessage === s"Job $jobId cancelled ") assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.contains(0)) assert(sparkListener.failedStages.size === 1) diff --git a/docs/configuration.md b/docs/configuration.md index 9c602402f0635..f3bfd036f4164 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -190,6 +190,13 @@ Apart from these, the following properties are also available, and may be useful user that started the Spark job has view access. + + spark.ui.killEnabled + true + + Allows stages and corresponding jobs to be killed from the web ui. + + spark.shuffle.compress true From 5cd11d51c19321981a6234a7765c7a5be6913433 Mon Sep 17 00:00:00 2001 From: Ivan Wick Date: Thu, 10 Apr 2014 17:49:30 -0700 Subject: [PATCH 051/641] Set spark.executor.uri from environment variable (needed by Mesos) The Mesos backend uses this property when setting up a slave process. It is similarly set in the Scala repl (org.apache.spark.repl.SparkILoop), but I couldn't find any analogous for pyspark. Author: Ivan Wick This patch had conflicts when merged, resolved by Committer: Matei Zaharia Closes #311 from ivanwick/master and squashes the following commits: da0c3e4 [Ivan Wick] Set spark.executor.uri from environment variable (needed by Mesos) --- python/pyspark/shell.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 35e48276e3cb9..61613dbed8dce 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -29,6 +29,9 @@ # this is the equivalent of ADD_JARS add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("ADD_FILES") != None else None +if os.environ.get("SPARK_EXECUTOR_URI"): + SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"]) + sc = SparkContext(os.environ.get("MASTER", "local[*]"), "PySparkShell", pyFiles=add_files) print """Welcome to From 7b4203ab4c640f7875ae3536228ed4d791062017 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 10 Apr 2014 18:25:54 -0700 Subject: [PATCH 052/641] Add Spark v0.9.1 to ec2 launch script and use it as the default Mainly ported from branch-0.9. Author: Harvey Feng Closes #385 from harveyfeng/0.9.1-ec2 and squashes the following commits: 769ac2f [Harvey Feng] Add Spark v0.9.1 to ec2 launch script and use it as the default --- ec2/spark_ec2.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index d8840c94ac17c..31209a662bbe1 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -70,7 +70,7 @@ def parse_args(): "slaves across multiple (an additional $0.01/Gb for bandwidth" + "between zones applies)") parser.add_option("-a", "--ami", help="Amazon Machine Image ID to use") - parser.add_option("-v", "--spark-version", default="0.9.0", + parser.add_option("-v", "--spark-version", default="0.9.1", help="Version of Spark to use: 'X.Y.Z' or a specific git hash") parser.add_option("--spark-git-repo", default="https://github.com/apache/spark", @@ -157,7 +157,7 @@ def is_active(instance): # Return correct versions of Spark and Shark, given the supplied Spark version def get_spark_shark_version(opts): - spark_shark_map = {"0.7.3": "0.7.1", "0.8.0": "0.8.0", "0.8.1": "0.8.1", "0.9.0": "0.9.0"} + spark_shark_map = {"0.7.3": "0.7.1", "0.8.0": "0.8.0", "0.8.1": "0.8.1", "0.9.0": "0.9.0", "0.9.1": "0.9.1"} version = opts.spark_version.replace("v", "") if version not in spark_shark_map: print >> stderr, "Don't know about Spark version: %s" % version From 44f654eecd3c181f2aeaff3871acf7f00eacc6b9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 10 Apr 2014 20:43:56 -0700 Subject: [PATCH 053/641] SPARK-1202: Improvements to task killing in the UI. 1. Adds a separate endpoint for the killing logic that is outside of a page. 2. Narrows the scope of the killingEnabled tracking. 3. Some style improvements. Author: Patrick Wendell Closes #386 from pwendell/kill-link and squashes the following commits: 8efe02b [Patrick Wendell] Improvements to task killing in the UI. --- .../org/apache/spark/ui/static/webui.css | 9 +++++++++ .../scala/org/apache/spark/ui/JettyUtils.scala | 2 ++ .../scala/org/apache/spark/ui/SparkUI.scala | 5 ++--- .../org/apache/spark/ui/jobs/IndexPage.scala | 11 ----------- .../apache/spark/ui/jobs/JobProgressUI.scala | 17 ++++++++++++++++- .../org/apache/spark/ui/jobs/StageTable.scala | 16 +++++++++------- 6 files changed, 38 insertions(+), 22 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index fe54c34ffb1da..599c3ac9b57c0 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -78,3 +78,12 @@ table.sortable thead { background-repeat: repeat-x; filter: progid:dximagetransform.microsoft.gradient(startColorstr='#FFA4EDFF', endColorstr='#FF94DDFF', GradientType=0); } + +span.kill-link { + margin-right: 2px; + color: gray; +} + +span.kill-link a { + color: gray; +} 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 9ce0398d010a8..dd0818e8ab01c 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -104,10 +104,12 @@ private[spark] object JettyUtils extends Logging { def createRedirectHandler( srcPath: String, destPath: String, + beforeRedirect: HttpServletRequest => Unit = x => (), basePath: String = ""): ServletContextHandler = { val prefixedDestPath = attachPrefix(basePath, destPath) val servlet = new HttpServlet { override def doGet(request: HttpServletRequest, response: HttpServletResponse) { + beforeRedirect(request) // Make sure we don't end up with "//" in the middle val newUrl = new URL(new URL(request.getRequestURL.toString), prefixedDestPath).toString response.sendRedirect(newUrl) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index dac11ec1cf52f..4c891d73afa87 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils /** Top level user interface for Spark */ private[spark] class SparkUI( val sc: SparkContext, - conf: SparkConf, + val conf: SparkConf, val listenerBus: SparkListenerBus, var appName: String, val basePath: String = "") @@ -46,7 +46,6 @@ private[spark] class SparkUI( val live = sc != null val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) - val killEnabled = conf.getBoolean("spark.ui.killEnabled", true) private val localHost = Utils.localHostName() private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) @@ -70,7 +69,7 @@ private[spark] class SparkUI( metricsServletHandlers ++ Seq[ServletContextHandler] ( createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"), - createRedirectHandler("/", "/stages", basePath) + createRedirectHandler("/", "/stages", basePath = basePath) ) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 5da5d1f2a3f45..8619a31380f1e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -32,7 +32,6 @@ private[ui] class IndexPage(parent: JobProgressUI) { private val sc = parent.sc private lazy val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler - private val killEnabled = parent.killEnabled private def appName = parent.appName @@ -43,16 +42,6 @@ private[ui] class IndexPage(parent: JobProgressUI) { val failedStages = listener.failedStages.reverse.toSeq val now = System.currentTimeMillis() - if (killEnabled) { - val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean - val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt - - if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) { - sc.cancelStage(stageId) - } - } - - val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent, parent.killEnabled) val completedStagesTable = diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index 9de659d6c7393..30e3f35f2182b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -32,7 +32,7 @@ private[ui] class JobProgressUI(parent: SparkUI) { val basePath = parent.basePath val live = parent.live val sc = parent.sc - val killEnabled = parent.killEnabled + val killEnabled = parent.conf.getBoolean("spark.ui.killEnabled", true) lazy val listener = _listener.get lazy val isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) @@ -51,7 +51,22 @@ private[ui] class JobProgressUI(parent: SparkUI) { def formatDuration(ms: Long) = Utils.msDurationToString(ms) + private def handleKillRequest(request: HttpServletRequest) = { + if (killEnabled) { + val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean + val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt + if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) { + sc.cancelStage(stageId) + } + // Do a quick pause here to give Spark time to kill the stage so it shows up as + // killed after the refresh. Note that this will block the serving thread so the + // time should be limited in duration. + Thread.sleep(100) + } + } + def getHandlers = Seq[ServletContextHandler]( + createRedirectHandler("/stages/stage/kill", "/stages", handleKillRequest), createServletHandler("/stages/stage", (request: HttpServletRequest) => stagePage.render(request), parent.securityManager, basePath), createServletHandler("/stages/pool", diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 1e874ae4969f9..e419fae5a6589 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -76,20 +76,22 @@ private[ui] class StageTable( } private def makeDescription(s: StageInfo): Seq[Node] = { + // scalastyle:off + val killLink = if (killEnabled) { + + (kill) + + } + // scalastyle:on + val nameLink = {s.name} - val killLink = if (killEnabled) { - - } val description = listener.stageIdToDescription.get(s.stageId) .map(d =>
    {d}
    {nameLink} {killLink}
    ) - .getOrElse(
    {nameLink} {killLink}
    ) + .getOrElse(
    {killLink}{nameLink}
    ) return description } From 446bb3417a2855a194d49acc0ac316a021eced9d Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 11 Apr 2014 13:17:48 +0530 Subject: [PATCH 054/641] SPARK-1417: Spark on Yarn - spark UI link from resourcemanager is broken Author: Thomas Graves Closes #344 from tgravescs/SPARK-1417 and squashes the following commits: c450b5f [Thomas Graves] fix test e1c1d7e [Thomas Graves] add missing $ to appUIAddress e982ddb [Thomas Graves] use appUIHostPort in appUIAddress 0803ec2 [Thomas Graves] Review comment updates - remove extra newline, simplify assert in test 658a8ec [Thomas Graves] Add a appUIHostPort routine 0614208 [Thomas Graves] Fix test 2a6b1b7 [Thomas Graves] SPARK-1417: Spark on Yarn - spark UI link from resourcemanager is broken --- .../scala/org/apache/spark/ui/SparkUI.scala | 7 +++- .../scala/org/apache/spark/SparkUISuite.scala | 35 +++++++++++++++++++ .../spark/deploy/yarn/ApplicationMaster.scala | 2 +- .../spark/deploy/yarn/ApplicationMaster.scala | 2 +- 4 files changed, 43 insertions(+), 3 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/SparkUISuite.scala diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 4c891d73afa87..7fa4fd3149eb6 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -113,7 +113,12 @@ private[spark] class SparkUI( logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) } - private[spark] def appUIAddress = "http://" + publicHost + ":" + boundPort + /** + * Return the application UI host:port. This does not include the scheme (http://). + */ + private[spark] def appUIHostPort = publicHost + ":" + boundPort + + private[spark] def appUIAddress = s"http://$appUIHostPort" } diff --git a/core/src/test/scala/org/apache/spark/SparkUISuite.scala b/core/src/test/scala/org/apache/spark/SparkUISuite.scala new file mode 100644 index 0000000000000..d0d119c15081d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SparkUISuite.scala @@ -0,0 +1,35 @@ +/* + * 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 + +import java.net.URI + +import org.scalatest.FunSuite + +class SparkUISuite extends FunSuite with SharedSparkContext { + + test("verify appUIAddress contains the scheme") { + val uiAddress = sc.ui.appUIAddress + assert(uiAddress.equals("http://" + sc.ui.appUIHostPort)) + } + + test("verify appUIAddress contains the port") { + val splitUIAddress = sc.ui.appUIAddress.split(':') + assert(splitUIAddress(2).toInt == sc.ui.boundPort) + } +} 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 910484ed5432a..67ec95c8fc04f 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 @@ -234,7 +234,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, assert(sparkContext != null || count >= numTries) if (null != sparkContext) { - uiAddress = sparkContext.ui.appUIAddress + uiAddress = sparkContext.ui.appUIHostPort this.yarnAllocator = YarnAllocationHandler.newAllocator( yarnConf, resourceManager, 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 c8a4d2e647cbd..61af0f9ac5ca0 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 @@ -220,7 +220,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, assert(sparkContext != null || numTries >= maxNumTries) if (sparkContext != null) { - uiAddress = sparkContext.ui.appUIAddress + uiAddress = sparkContext.ui.appUIHostPort this.yarnAllocator = YarnAllocationHandler.newAllocator( yarnConf, amClient, From 98225a6effd077a1b97c7e485d45ffd89b2c5b7f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 11 Apr 2014 10:45:27 -0700 Subject: [PATCH 055/641] Some clean up in build/docs (a) Deleted an outdated line from the docs (b) Removed a work around that is no longer necessary given the mesos version bump. Author: Patrick Wendell Closes #382 from pwendell/maven-clean and squashes the following commits: f0447fa [Patrick Wendell] Minor doc clean-up --- docs/index.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/index.md b/docs/index.md index 7a13fa9a9a2b6..89ec5b05488a9 100644 --- a/docs/index.md +++ b/docs/index.md @@ -67,8 +67,6 @@ In addition, if you wish to run Spark on [YARN](running-on-yarn.html), set Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`. -For this version of Spark (0.8.1) Hadoop 2.2.x (or newer) users will have to build Spark and publish it locally. See [Launching Spark on YARN](running-on-yarn.html). This is needed because Hadoop 2.2 has non backwards compatible API changes. - # Where to Go from Here **Programming guides:** From f5ace8da34c58d1005c7c377cfe3df21102c1dd6 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 11 Apr 2014 12:06:13 -0700 Subject: [PATCH 056/641] [SPARK-1225, 1241] [MLLIB] Add AreaUnderCurve and BinaryClassificationMetrics This PR implements a generic version of `AreaUnderCurve` using the `RDD.sliding` implementation from https://github.com/apache/spark/pull/136 . It also contains refactoring of https://github.com/apache/spark/pull/160 for binary classification evaluation. Author: Xiangrui Meng Closes #364 from mengxr/auc and squashes the following commits: a05941d [Xiangrui Meng] replace TP/FP/TN/FN by their full names 3f42e98 [Xiangrui Meng] add (0, 0), (1, 1) to roc, and (0, 1) to pr fb4b6d2 [Xiangrui Meng] rename Evaluator to Metrics and add more metrics b1b7dab [Xiangrui Meng] fix code styles 9dc3518 [Xiangrui Meng] add tests for BinaryClassificationEvaluator ca31da5 [Xiangrui Meng] remove PredictionAndResponse 3d71525 [Xiangrui Meng] move binary evalution classes to evaluation.binary 8f78958 [Xiangrui Meng] add PredictionAndResponse dda82d5 [Xiangrui Meng] add confusion matrix aa7e278 [Xiangrui Meng] add initial version of binary classification evaluator 221ebce [Xiangrui Meng] add a new test to sliding a920865 [Xiangrui Meng] Merge branch 'sliding' into auc a9b250a [Xiangrui Meng] move sliding to mllib cab9a52 [Xiangrui Meng] use last for the last element db6cb30 [Xiangrui Meng] remove unnecessary toSeq 9916202 [Xiangrui Meng] change RDD.sliding return type to RDD[Seq[T]] 284d991 [Xiangrui Meng] change SlidedRDD to SlidingRDD c1c6c22 [Xiangrui Meng] add AreaUnderCurve 65461b2 [Xiangrui Meng] Merge branch 'sliding' into auc 5ee6001 [Xiangrui Meng] add TODO d2a600d [Xiangrui Meng] add sliding to rdd --- .../mllib/evaluation/AreaUnderCurve.scala | 62 ++++++ .../BinaryClassificationMetricComputers.scala | 57 +++++ .../binary/BinaryClassificationMetrics.scala | 204 ++++++++++++++++++ .../binary/BinaryConfusionMatrix.scala | 41 ++++ .../apache/spark/mllib/rdd/RDDFunctions.scala | 53 +++++ .../apache/spark/mllib/rdd/SlidingRDD.scala | 104 +++++++++ .../evaluation/AreaUnderCurveSuite.scala | 46 ++++ .../BinaryClassificationMetricsSuite.scala | 55 +++++ .../spark/mllib/rdd/RDDFunctionsSuite.scala | 49 +++++ 9 files changed, 671 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala new file mode 100644 index 0000000000000..7858ec602483f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation + +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.rdd.RDDFunctions._ + +/** + * Computes the area under the curve (AUC) using the trapezoidal rule. + */ +private[evaluation] object AreaUnderCurve { + + /** + * Uses the trapezoidal rule to compute the area under the line connecting the two input points. + * @param points two 2D points stored in Seq + */ + private def trapezoid(points: Seq[(Double, Double)]): Double = { + require(points.length == 2) + val x = points.head + val y = points.last + (y._1 - x._1) * (y._2 + x._2) / 2.0 + } + + /** + * Returns the area under the given curve. + * + * @param curve a RDD of ordered 2D points stored in pairs representing a curve + */ + def of(curve: RDD[(Double, Double)]): Double = { + curve.sliding(2).aggregate(0.0)( + seqOp = (auc: Double, points: Seq[(Double, Double)]) => auc + trapezoid(points), + combOp = _ + _ + ) + } + + /** + * Returns the area under the given curve. + * + * @param curve an iterator over ordered 2D points stored in pairs representing a curve + */ + def of(curve: Iterable[(Double, Double)]): Double = { + curve.toIterator.sliding(2).withPartial(false).aggregate(0.0)( + seqop = (auc: Double, points: Seq[(Double, Double)]) => auc + trapezoid(points), + combop = _ + _ + ) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala new file mode 100644 index 0000000000000..562663ad36b40 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation.binary + +/** + * Trait for a binary classification evaluation metric computer. + */ +private[evaluation] trait BinaryClassificationMetricComputer extends Serializable { + def apply(c: BinaryConfusionMatrix): Double +} + +/** Precision. */ +private[evaluation] object Precision extends BinaryClassificationMetricComputer { + override def apply(c: BinaryConfusionMatrix): Double = + c.numTruePositives.toDouble / (c.numTruePositives + c.numFalsePositives) +} + +/** False positive rate. */ +private[evaluation] object FalsePositiveRate extends BinaryClassificationMetricComputer { + override def apply(c: BinaryConfusionMatrix): Double = + c.numFalsePositives.toDouble / c.numNegatives +} + +/** Recall. */ +private[evaluation] object Recall extends BinaryClassificationMetricComputer { + override def apply(c: BinaryConfusionMatrix): Double = + c.numTruePositives.toDouble / c.numPositives +} + +/** + * F-Measure. + * @param beta the beta constant in F-Measure + * @see http://en.wikipedia.org/wiki/F1_score + */ +private[evaluation] case class FMeasure(beta: Double) extends BinaryClassificationMetricComputer { + private val beta2 = beta * beta + override def apply(c: BinaryConfusionMatrix): Double = { + val precision = Precision(c) + val recall = Recall(c) + (1.0 + beta2) * (precision * recall) / (beta2 * precision + recall) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala new file mode 100644 index 0000000000000..ed7b0fc943367 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation.binary + +import org.apache.spark.rdd.{UnionRDD, RDD} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.evaluation.AreaUnderCurve +import org.apache.spark.Logging + +/** + * Implementation of [[org.apache.spark.mllib.evaluation.binary.BinaryConfusionMatrix]]. + * + * @param count label counter for labels with scores greater than or equal to the current score + * @param totalCount label counter for all labels + */ +private case class BinaryConfusionMatrixImpl( + count: LabelCounter, + totalCount: LabelCounter) extends BinaryConfusionMatrix with Serializable { + + /** number of true positives */ + override def numTruePositives: Long = count.numPositives + + /** number of false positives */ + override def numFalsePositives: Long = count.numNegatives + + /** number of false negatives */ + override def numFalseNegatives: Long = totalCount.numPositives - count.numPositives + + /** number of true negatives */ + override def numTrueNegatives: Long = totalCount.numNegatives - count.numNegatives + + /** number of positives */ + override def numPositives: Long = totalCount.numPositives + + /** number of negatives */ + override def numNegatives: Long = totalCount.numNegatives +} + +/** + * Evaluator for binary classification. + * + * @param scoreAndLabels an RDD of (score, label) pairs. + */ +class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) + extends Serializable with Logging { + + private lazy val ( + cumulativeCounts: RDD[(Double, LabelCounter)], + confusions: RDD[(Double, BinaryConfusionMatrix)]) = { + // Create a bin for each distinct score value, count positives and negatives within each bin, + // and then sort by score values in descending order. + val counts = scoreAndLabels.combineByKey( + createCombiner = (label: Double) => new LabelCounter(0L, 0L) += label, + mergeValue = (c: LabelCounter, label: Double) => c += label, + mergeCombiners = (c1: LabelCounter, c2: LabelCounter) => c1 += c2 + ).sortByKey(ascending = false) + val agg = counts.values.mapPartitions({ iter => + val agg = new LabelCounter() + iter.foreach(agg += _) + Iterator(agg) + }, preservesPartitioning = true).collect() + val partitionwiseCumulativeCounts = + agg.scanLeft(new LabelCounter())((agg: LabelCounter, c: LabelCounter) => agg.clone() += c) + val totalCount = partitionwiseCumulativeCounts.last + logInfo(s"Total counts: $totalCount") + val cumulativeCounts = counts.mapPartitionsWithIndex( + (index: Int, iter: Iterator[(Double, LabelCounter)]) => { + val cumCount = partitionwiseCumulativeCounts(index) + iter.map { case (score, c) => + cumCount += c + (score, cumCount.clone()) + } + }, preservesPartitioning = true) + cumulativeCounts.persist() + val confusions = cumulativeCounts.map { case (score, cumCount) => + (score, BinaryConfusionMatrixImpl(cumCount, totalCount).asInstanceOf[BinaryConfusionMatrix]) + } + (cumulativeCounts, confusions) + } + + /** Unpersist intermediate RDDs used in the computation. */ + def unpersist() { + cumulativeCounts.unpersist() + } + + /** Returns thresholds in descending order. */ + def thresholds(): RDD[Double] = cumulativeCounts.map(_._1) + + /** + * Returns the receiver operating characteristic (ROC) curve, + * which is an RDD of (false positive rate, true positive rate) + * with (0.0, 0.0) prepended and (1.0, 1.0) appended to it. + * @see http://en.wikipedia.org/wiki/Receiver_operating_characteristic + */ + def roc(): RDD[(Double, Double)] = { + val rocCurve = createCurve(FalsePositiveRate, Recall) + val sc = confusions.context + val first = sc.makeRDD(Seq((0.0, 0.0)), 1) + val last = sc.makeRDD(Seq((1.0, 1.0)), 1) + new UnionRDD[(Double, Double)](sc, Seq(first, rocCurve, last)) + } + + /** + * Computes the area under the receiver operating characteristic (ROC) curve. + */ + def areaUnderROC(): Double = AreaUnderCurve.of(roc()) + + /** + * Returns the precision-recall curve, which is an RDD of (recall, precision), + * NOT (precision, recall), with (0.0, 1.0) prepended to it. + * @see http://en.wikipedia.org/wiki/Precision_and_recall + */ + def pr(): RDD[(Double, Double)] = { + val prCurve = createCurve(Recall, Precision) + val sc = confusions.context + val first = sc.makeRDD(Seq((0.0, 1.0)), 1) + first.union(prCurve) + } + + /** + * Computes the area under the precision-recall curve. + */ + def areaUnderPR(): Double = AreaUnderCurve.of(pr()) + + /** + * Returns the (threshold, F-Measure) curve. + * @param beta the beta factor in F-Measure computation. + * @return an RDD of (threshold, F-Measure) pairs. + * @see http://en.wikipedia.org/wiki/F1_score + */ + def fMeasureByThreshold(beta: Double): RDD[(Double, Double)] = createCurve(FMeasure(beta)) + + /** Returns the (threshold, F-Measure) curve with beta = 1.0. */ + def fMeasureByThreshold(): RDD[(Double, Double)] = fMeasureByThreshold(1.0) + + /** Returns the (threshold, precision) curve. */ + def precisionByThreshold(): RDD[(Double, Double)] = createCurve(Precision) + + /** Returns the (threshold, recall) curve. */ + def recallByThreshold(): RDD[(Double, Double)] = createCurve(Recall) + + /** Creates a curve of (threshold, metric). */ + private def createCurve(y: BinaryClassificationMetricComputer): RDD[(Double, Double)] = { + confusions.map { case (s, c) => + (s, y(c)) + } + } + + /** Creates a curve of (metricX, metricY). */ + private def createCurve( + x: BinaryClassificationMetricComputer, + y: BinaryClassificationMetricComputer): RDD[(Double, Double)] = { + confusions.map { case (_, c) => + (x(c), y(c)) + } + } +} + +/** + * A counter for positives and negatives. + * + * @param numPositives number of positive labels + * @param numNegatives number of negative labels + */ +private class LabelCounter( + var numPositives: Long = 0L, + var numNegatives: Long = 0L) extends Serializable { + + /** Processes a label. */ + def +=(label: Double): LabelCounter = { + // Though we assume 1.0 for positive and 0.0 for negative, the following check will handle + // -1.0 for negative as well. + if (label > 0.5) numPositives += 1L else numNegatives += 1L + this + } + + /** Merges another counter. */ + def +=(other: LabelCounter): LabelCounter = { + numPositives += other.numPositives + numNegatives += other.numNegatives + this + } + + override def clone: LabelCounter = { + new LabelCounter(numPositives, numNegatives) + } + + override def toString: String = s"{numPos: $numPositives, numNeg: $numNegatives}" +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala new file mode 100644 index 0000000000000..75a75b216002a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation.binary + +/** + * Trait for a binary confusion matrix. + */ +private[evaluation] trait BinaryConfusionMatrix { + /** number of true positives */ + def numTruePositives: Long + + /** number of false positives */ + def numFalsePositives: Long + + /** number of false negatives */ + def numFalseNegatives: Long + + /** number of true negatives */ + def numTrueNegatives: Long + + /** number of positives */ + def numPositives: Long = numTruePositives + numFalseNegatives + + /** number of negatives */ + def numNegatives: Long = numFalsePositives + numTrueNegatives +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala new file mode 100644 index 0000000000000..873de871fd884 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.rdd + +import scala.reflect.ClassTag + +import org.apache.spark.rdd.RDD + +/** + * Machine learning specific RDD functions. + */ +private[mllib] +class RDDFunctions[T: ClassTag](self: RDD[T]) { + + /** + * Returns a RDD from grouping items of its parent RDD in fixed size blocks by passing a sliding + * window over them. The ordering is first based on the partition index and then the ordering of + * items within each partition. This is similar to sliding in Scala collections, except that it + * becomes an empty RDD if the window size is greater than the total number of items. It needs to + * trigger a Spark job if the parent RDD has more than one partitions and the window size is + * greater than 1. + */ + def sliding(windowSize: Int): RDD[Seq[T]] = { + require(windowSize > 0, s"Sliding window size must be positive, but got $windowSize.") + if (windowSize == 1) { + self.map(Seq(_)) + } else { + new SlidingRDD[T](self, windowSize) + } + } +} + +private[mllib] +object RDDFunctions { + + /** Implicit conversion from an RDD to RDDFunctions. */ + implicit def fromRDD[T: ClassTag](rdd: RDD[T]) = new RDDFunctions[T](rdd) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala new file mode 100644 index 0000000000000..dd80782c0f001 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.rdd + +import scala.collection.mutable +import scala.reflect.ClassTag + +import org.apache.spark.{TaskContext, Partition} +import org.apache.spark.rdd.RDD + +private[mllib] +class SlidingRDDPartition[T](val idx: Int, val prev: Partition, val tail: Seq[T]) + extends Partition with Serializable { + override val index: Int = idx +} + +/** + * Represents a RDD from grouping items of its parent RDD in fixed size blocks by passing a sliding + * window over them. The ordering is first based on the partition index and then the ordering of + * items within each partition. This is similar to sliding in Scala collections, except that it + * becomes an empty RDD if the window size is greater than the total number of items. It needs to + * trigger a Spark job if the parent RDD has more than one partitions. To make this operation + * efficient, the number of items per partition should be larger than the window size and the + * window size should be small, e.g., 2. + * + * @param parent the parent RDD + * @param windowSize the window size, must be greater than 1 + * + * @see [[org.apache.spark.mllib.rdd.RDDFunctions#sliding]] + */ +private[mllib] +class SlidingRDD[T: ClassTag](@transient val parent: RDD[T], val windowSize: Int) + extends RDD[Seq[T]](parent) { + + require(windowSize > 1, s"Window size must be greater than 1, but got $windowSize.") + + override def compute(split: Partition, context: TaskContext): Iterator[Seq[T]] = { + val part = split.asInstanceOf[SlidingRDDPartition[T]] + (firstParent[T].iterator(part.prev, context) ++ part.tail) + .sliding(windowSize) + .withPartial(false) + } + + override def getPreferredLocations(split: Partition): Seq[String] = + firstParent[T].preferredLocations(split.asInstanceOf[SlidingRDDPartition[T]].prev) + + override def getPartitions: Array[Partition] = { + val parentPartitions = parent.partitions + val n = parentPartitions.size + if (n == 0) { + Array.empty + } else if (n == 1) { + Array(new SlidingRDDPartition[T](0, parentPartitions(0), Seq.empty)) + } else { + val n1 = n - 1 + val w1 = windowSize - 1 + // Get the first w1 items of each partition, starting from the second partition. + val nextHeads = + parent.context.runJob(parent, (iter: Iterator[T]) => iter.take(w1).toArray, 1 until n, true) + val partitions = mutable.ArrayBuffer[SlidingRDDPartition[T]]() + var i = 0 + var partitionIndex = 0 + while (i < n1) { + var j = i + val tail = mutable.ListBuffer[T]() + // Keep appending to the current tail until appended a head of size w1. + while (j < n1 && nextHeads(j).size < w1) { + tail ++= nextHeads(j) + j += 1 + } + if (j < n1) { + tail ++= nextHeads(j) + j += 1 + } + partitions += new SlidingRDDPartition[T](partitionIndex, parentPartitions(i), tail) + partitionIndex += 1 + // Skip appended heads. + i = j + } + // If the head of last partition has size w1, we also need to add this partition. + if (nextHeads.last.size == w1) { + partitions += new SlidingRDDPartition[T](partitionIndex, parentPartitions(n1), Seq.empty) + } + partitions.toArray + } + } + + // TODO: Override methods such as aggregate, which only requires one Spark job. +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala new file mode 100644 index 0000000000000..1c9844f289fe0 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.LocalSparkContext + +class AreaUnderCurveSuite extends FunSuite with LocalSparkContext { + test("auc computation") { + val curve = Seq((0.0, 0.0), (1.0, 1.0), (2.0, 3.0), (3.0, 0.0)) + val auc = 4.0 + assert(AreaUnderCurve.of(curve) === auc) + val rddCurve = sc.parallelize(curve, 2) + assert(AreaUnderCurve.of(rddCurve) == auc) + } + + test("auc of an empty curve") { + val curve = Seq.empty[(Double, Double)] + assert(AreaUnderCurve.of(curve) === 0.0) + val rddCurve = sc.parallelize(curve, 2) + assert(AreaUnderCurve.of(rddCurve) === 0.0) + } + + test("auc of a curve with a single point") { + val curve = Seq((1.0, 1.0)) + assert(AreaUnderCurve.of(curve) === 0.0) + val rddCurve = sc.parallelize(curve, 2) + assert(AreaUnderCurve.of(rddCurve) === 0.0) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala new file mode 100644 index 0000000000000..173fdaefab3da --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation.binary + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.evaluation.AreaUnderCurve + +class BinaryClassificationMetricsSuite extends FunSuite with LocalSparkContext { + test("binary evaluation metrics") { + val scoreAndLabels = sc.parallelize( + Seq((0.1, 0.0), (0.1, 1.0), (0.4, 0.0), (0.6, 0.0), (0.6, 1.0), (0.6, 1.0), (0.8, 1.0)), 2) + val metrics = new BinaryClassificationMetrics(scoreAndLabels) + val threshold = Seq(0.8, 0.6, 0.4, 0.1) + val numTruePositives = Seq(1, 3, 3, 4) + val numFalsePositives = Seq(0, 1, 2, 3) + val numPositives = 4 + val numNegatives = 3 + val precision = numTruePositives.zip(numFalsePositives).map { case (t, f) => + t.toDouble / (t + f) + } + val recall = numTruePositives.map(t => t.toDouble / numPositives) + val fpr = numFalsePositives.map(f => f.toDouble / numNegatives) + val rocCurve = Seq((0.0, 0.0)) ++ fpr.zip(recall) ++ Seq((1.0, 1.0)) + val pr = recall.zip(precision) + val prCurve = Seq((0.0, 1.0)) ++ pr + val f1 = pr.map { case (r, p) => 2.0 * (p * r) / (p + r) } + val f2 = pr.map { case (r, p) => 5.0 * (p * r) / (4.0 * p + r)} + assert(metrics.thresholds().collect().toSeq === threshold) + assert(metrics.roc().collect().toSeq === rocCurve) + assert(metrics.areaUnderROC() === AreaUnderCurve.of(rocCurve)) + assert(metrics.pr().collect().toSeq === prCurve) + assert(metrics.areaUnderPR() === AreaUnderCurve.of(prCurve)) + assert(metrics.fMeasureByThreshold().collect().toSeq === threshold.zip(f1)) + assert(metrics.fMeasureByThreshold(2.0).collect().toSeq === threshold.zip(f2)) + assert(metrics.precisionByThreshold().collect().toSeq === threshold.zip(precision)) + assert(metrics.recallByThreshold().collect().toSeq === threshold.zip(recall)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala new file mode 100644 index 0000000000000..3f3b10dfff35e --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.rdd + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.rdd.RDDFunctions._ + +class RDDFunctionsSuite extends FunSuite with LocalSparkContext { + + test("sliding") { + val data = 0 until 6 + for (numPartitions <- 1 to 8) { + val rdd = sc.parallelize(data, numPartitions) + for (windowSize <- 1 to 6) { + val sliding = rdd.sliding(windowSize).collect().map(_.toList).toList + val expected = data.sliding(windowSize).map(_.toList).toList + assert(sliding === expected) + } + assert(rdd.sliding(7).collect().isEmpty, + "Should return an empty RDD if the window size is greater than the number of items.") + } + } + + test("sliding with empty partitions") { + val data = Seq(Seq(1, 2, 3), Seq.empty[Int], Seq(4), Seq.empty[Int], Seq(5, 6, 7)) + val rdd = sc.parallelize(data, data.length).flatMap(s => s) + assert(rdd.partitions.size === data.length) + val sliding = rdd.sliding(3) + val expected = data.flatMap(x => x).sliding(3).toList + assert(sliding.collect().toList === expected) + } +} From 6a0f8e35ce7595c4ece11fe04133fd44ffbe5b06 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 11 Apr 2014 13:23:21 -0700 Subject: [PATCH 057/641] HOTFIX: Ignore python metastore files in RAT checks. This was causing some errors with pull request tests. Author: Patrick Wendell Closes #393 from pwendell/hotfix and squashes the following commits: 6201dd3 [Patrick Wendell] HOTFIX: Ignore python metastore files in RAT checks. --- .rat-excludes | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.rat-excludes b/.rat-excludes index a2b5665a0be26..8954330bd10a7 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -39,4 +39,6 @@ work .*\.q golden test.out/* -.*iml \ No newline at end of file +.*iml +python/metastore/service.properties +python/metastore/db.lck From 7038b00be9c84a4d92f9d95ff3d75fae47d57d87 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 11 Apr 2014 19:41:40 -0700 Subject: [PATCH 058/641] [FIX] make coalesce test deterministic in RDDSuite Make coalesce test deterministic by setting pre-defined seeds. (Saw random failures in other PRs.) Author: Xiangrui Meng Closes #387 from mengxr/fix-random and squashes the following commits: 59bc16f [Xiangrui Meng] make coalesce test deterministic in RDDSuite --- .../scala/org/apache/spark/rdd/RDDSuite.scala | 61 ++++++++++--------- 1 file changed, 33 insertions(+), 28 deletions(-) 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 25973348a7837..1901330d8b188 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -274,37 +274,42 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("coalesced RDDs with locality, large scale (10K partitions)") { // large scale experiment import collection.mutable - val rnd = scala.util.Random val partitions = 10000 val numMachines = 50 val machines = mutable.ListBuffer[String]() - (1 to numMachines).foreach(machines += "m"+_) - - val blocks = (1 to partitions).map(i => - { (i, Array.fill(3)(machines(rnd.nextInt(machines.size))).toList) } ) - - val data2 = sc.makeRDD(blocks) - val coalesced2 = data2.coalesce(numMachines*2) - - // test that you get over 90% locality in each group - val minLocality = coalesced2.partitions - .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) - .foldLeft(1.0)((perc, loc) => math.min(perc,loc)) - assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.0).toInt + "%") - - // test that the groups are load balanced with 100 +/- 20 elements in each - val maxImbalance = coalesced2.partitions - .map(part => part.asInstanceOf[CoalescedRDDPartition].parents.size) - .foldLeft(0)((dev, curr) => math.max(math.abs(100-curr),dev)) - assert(maxImbalance <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance) - - val data3 = sc.makeRDD(blocks).map(i => i*2) // derived RDD to test *current* pref locs - val coalesced3 = data3.coalesce(numMachines*2) - val minLocality2 = coalesced3.partitions - .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) - .foldLeft(1.0)((perc, loc) => math.min(perc,loc)) - assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " + - (minLocality2*100.0).toInt + "%") + (1 to numMachines).foreach(machines += "m" + _) + val rnd = scala.util.Random + for (seed <- 1 to 5) { + rnd.setSeed(seed) + + val blocks = (1 to partitions).map { i => + (i, Array.fill(3)(machines(rnd.nextInt(machines.size))).toList) + } + + val data2 = sc.makeRDD(blocks) + val coalesced2 = data2.coalesce(numMachines * 2) + + // test that you get over 90% locality in each group + val minLocality = coalesced2.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) + .foldLeft(1.0)((perc, loc) => math.min(perc, loc)) + assert(minLocality >= 0.90, "Expected 90% locality but got " + + (minLocality * 100.0).toInt + "%") + + // test that the groups are load balanced with 100 +/- 20 elements in each + val maxImbalance = coalesced2.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].parents.size) + .foldLeft(0)((dev, curr) => math.max(math.abs(100 - curr), dev)) + assert(maxImbalance <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance) + + val data3 = sc.makeRDD(blocks).map(i => i * 2) // derived RDD to test *current* pref locs + val coalesced3 = data3.coalesce(numMachines * 2) + val minLocality2 = coalesced3.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) + .foldLeft(1.0)((perc, loc) => math.min(perc, loc)) + assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " + + (minLocality2 * 100.0).toInt + "%") + } } test("zipped RDDs") { From fdfb45e691946f3153d6c696bec6d7f3e391e301 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Fri, 11 Apr 2014 19:43:22 -0700 Subject: [PATCH 059/641] [WIP] [SPARK-1328] Add vector statistics As with the new vector system in MLlib, we find that it is good to add some new APIs to precess the `RDD[Vector]`. Beside, the former implementation of `computeStat` is not stable which could loss precision, and has the possibility to cause `Nan` in scientific computing, just as said in the [SPARK-1328](https://spark-project.atlassian.net/browse/SPARK-1328). APIs contain: * rowMeans(): RDD[Double] * rowNorm2(): RDD[Double] * rowSDs(): RDD[Double] * colMeans(): Vector * colMeans(size: Int): Vector * colNorm2(): Vector * colNorm2(size: Int): Vector * colSDs(): Vector * colSDs(size: Int): Vector * maxOption((Vector, Vector) => Boolean): Option[Vector] * minOption((Vector, Vector) => Boolean): Option[Vector] * rowShrink(): RDD[Vector] * colShrink(): RDD[Vector] This is working in process now, and some more APIs will add to `LabeledPoint`. Moreover, the implicit declaration will move from `MLUtils` to `MLContext` later. Author: Xusen Yin Author: Xiangrui Meng Closes #268 from yinxusen/vector-statistics and squashes the following commits: d61363f [Xusen Yin] rebase to latest master 16ae684 [Xusen Yin] fix minor error and remove useless method 10cf5d3 [Xusen Yin] refine some return type b064714 [Xusen Yin] remove computeStat in MLUtils cbbefdb [Xiangrui Meng] update multivariate statistical summary interface and clean tests 4eaf28a [Xusen Yin] merge VectorRDDStatistics into RowMatrix 48ee053 [Xusen Yin] fix minor error e624f93 [Xusen Yin] fix scala style error 1fba230 [Xusen Yin] merge while loop together 69e1f37 [Xusen Yin] remove lazy eval, and minor memory footprint 548e9de [Xusen Yin] minor revision 86522c4 [Xusen Yin] add comments on functions dc77e38 [Xusen Yin] test sparse vector RDD 18cf072 [Xusen Yin] change def to lazy val to make sure that the computations in function be evaluated only once f7a3ca2 [Xusen Yin] fix the corner case of maxmin 967d041 [Xusen Yin] full revision with Aggregator class 138300c [Xusen Yin] add new Aggregator class 1376ff4 [Xusen Yin] rename variables and adjust code 4a5c38d [Xusen Yin] add scala doc, refine code and comments 036b7a5 [Xusen Yin] fix the bug of Nan occur f6e8e9a [Xusen Yin] add sparse vectors test 4cfbadf [Xusen Yin] fix bug of min max 4e4fbd1 [Xusen Yin] separate seqop and combop out as independent functions a6d5a2e [Xusen Yin] rewrite for only computing non-zero elements 3980287 [Xusen Yin] rename variables 62a2c3e [Xusen Yin] use axpy and in-place if possible 9a75ebd [Xusen Yin] add case class to wrap return values d816ac7 [Xusen Yin] remove useless APIs c4651bb [Xusen Yin] remove row-wise APIs and refine code 1338ea1 [Xusen Yin] all-in-one version test passed cc65810 [Xusen Yin] add parallel mean and variance 9af2e95 [Xusen Yin] refine the code style ad6c82d [Xusen Yin] add shrink test e09d5d2 [Xusen Yin] add scala docs and refine shrink method 8ef3377 [Xusen Yin] pass all tests 28cf060 [Xusen Yin] fix error of column means 54b19ab [Xusen Yin] add new API to shrink RDD[Vector] 8c6c0e1 [Xusen Yin] add basic statistics --- .../mllib/linalg/distributed/RowMatrix.scala | 165 +++++++++++++++++- .../stat/MultivariateStatisticalSummary.scala | 56 ++++++ .../org/apache/spark/mllib/util/MLUtils.scala | 57 +----- .../linalg/distributed/RowMatrixSuite.scala | 15 ++ .../spark/mllib/util/MLUtilsSuite.scala | 13 -- 5 files changed, 230 insertions(+), 76 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index f65f43dd3007b..0c0afcd9ec0d7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.linalg.distributed import java.util -import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, svd => brzSvd} +import breeze.linalg.{Vector => BV, DenseMatrix => BDM, DenseVector => BDV, svd => brzSvd} import breeze.numerics.{sqrt => brzSqrt} import com.github.fommil.netlib.BLAS.{getInstance => blas} @@ -27,6 +27,138 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg._ import org.apache.spark.rdd.RDD import org.apache.spark.Logging +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary + +/** + * Column statistics aggregator implementing + * [[org.apache.spark.mllib.stat.MultivariateStatisticalSummary]] + * together with add() and merge() function. + * A numerically stable algorithm is implemented to compute sample mean and variance: + *[[http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance variance-wiki]]. + * Zero elements (including explicit zero values) are skipped when calling add() and merge(), + * to have time complexity O(nnz) instead of O(n) for each column. + */ +private class ColumnStatisticsAggregator(private val n: Int) + extends MultivariateStatisticalSummary with Serializable { + + private val currMean: BDV[Double] = BDV.zeros[Double](n) + private val currM2n: BDV[Double] = BDV.zeros[Double](n) + private var totalCnt = 0.0 + private val nnz: BDV[Double] = BDV.zeros[Double](n) + private val currMax: BDV[Double] = BDV.fill(n)(Double.MinValue) + private val currMin: BDV[Double] = BDV.fill(n)(Double.MaxValue) + + override def mean: Vector = { + val realMean = BDV.zeros[Double](n) + var i = 0 + while (i < n) { + realMean(i) = currMean(i) * nnz(i) / totalCnt + i += 1 + } + Vectors.fromBreeze(realMean) + } + + override def variance: Vector = { + val realVariance = BDV.zeros[Double](n) + + val denominator = totalCnt - 1.0 + + // Sample variance is computed, if the denominator is less than 0, the variance is just 0. + if (denominator > 0.0) { + val deltaMean = currMean + var i = 0 + while (i < currM2n.size) { + realVariance(i) = + currM2n(i) + deltaMean(i) * deltaMean(i) * nnz(i) * (totalCnt - nnz(i)) / totalCnt + realVariance(i) /= denominator + i += 1 + } + } + + Vectors.fromBreeze(realVariance) + } + + override def count: Long = totalCnt.toLong + + override def numNonzeros: Vector = Vectors.fromBreeze(nnz) + + override def max: Vector = { + var i = 0 + while (i < n) { + if ((nnz(i) < totalCnt) && (currMax(i) < 0.0)) currMax(i) = 0.0 + i += 1 + } + Vectors.fromBreeze(currMax) + } + + override def min: Vector = { + var i = 0 + while (i < n) { + if ((nnz(i) < totalCnt) && (currMin(i) > 0.0)) currMin(i) = 0.0 + i += 1 + } + Vectors.fromBreeze(currMin) + } + + /** + * Aggregates a row. + */ + def add(currData: BV[Double]): this.type = { + currData.activeIterator.foreach { + case (_, 0.0) => // Skip explicit zero elements. + case (i, value) => + if (currMax(i) < value) { + currMax(i) = value + } + if (currMin(i) > value) { + currMin(i) = value + } + + val tmpPrevMean = currMean(i) + currMean(i) = (currMean(i) * nnz(i) + value) / (nnz(i) + 1.0) + currM2n(i) += (value - currMean(i)) * (value - tmpPrevMean) + + nnz(i) += 1.0 + } + + totalCnt += 1.0 + this + } + + /** + * Merges another aggregator. + */ + def merge(other: ColumnStatisticsAggregator): this.type = { + require(n == other.n, s"Dimensions mismatch. Expecting $n but got ${other.n}.") + + totalCnt += other.totalCnt + val deltaMean = currMean - other.currMean + + var i = 0 + while (i < n) { + // merge mean together + if (other.currMean(i) != 0.0) { + currMean(i) = (currMean(i) * nnz(i) + other.currMean(i) * other.nnz(i)) / + (nnz(i) + other.nnz(i)) + } + // merge m2n together + if (nnz(i) + other.nnz(i) != 0.0) { + currM2n(i) += other.currM2n(i) + deltaMean(i) * deltaMean(i) * nnz(i) * other.nnz(i) / + (nnz(i) + other.nnz(i)) + } + if (currMax(i) < other.currMax(i)) { + currMax(i) = other.currMax(i) + } + if (currMin(i) > other.currMin(i)) { + currMin(i) = other.currMin(i) + } + i += 1 + } + + nnz += other.nnz + this + } +} /** * :: Experimental :: @@ -182,13 +314,7 @@ class RowMatrix( combOp = (s1: (Long, BDV[Double]), s2: (Long, BDV[Double])) => (s1._1 + s2._1, s1._2 += s2._2) ) - // Update _m if it is not set, or verify its value. - if (nRows <= 0L) { - nRows = m - } else { - require(nRows == m, - s"The number of rows $m is different from what specified or previously computed: ${nRows}.") - } + updateNumRows(m) mean :/= m.toDouble @@ -240,6 +366,19 @@ class RowMatrix( } } + /** + * Computes column-wise summary statistics. + */ + def computeColumnSummaryStatistics(): MultivariateStatisticalSummary = { + val zeroValue = new ColumnStatisticsAggregator(numCols().toInt) + val summary = rows.map(_.toBreeze).aggregate[ColumnStatisticsAggregator](zeroValue)( + (aggregator, data) => aggregator.add(data), + (aggregator1, aggregator2) => aggregator1.merge(aggregator2) + ) + updateNumRows(summary.count) + summary + } + /** * Multiply this matrix by a local matrix on the right. * @@ -276,6 +415,16 @@ class RowMatrix( } mat } + + /** Updates or verfires the number of rows. */ + private def updateNumRows(m: Long) { + if (nRows <= 0) { + nRows == m + } else { + require(nRows == m, + s"The number of rows $m is different from what specified or previously computed: ${nRows}.") + } + } } object RowMatrix { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala new file mode 100644 index 0000000000000..f9eb343da2b82 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import org.apache.spark.mllib.linalg.Vector + +/** + * Trait for multivariate statistical summary of a data matrix. + */ +trait MultivariateStatisticalSummary { + + /** + * Sample mean vector. + */ + def mean: Vector + + /** + * Sample variance vector. Should return a zero vector if the sample size is 1. + */ + def variance: Vector + + /** + * Sample size. + */ + def count: Long + + /** + * Number of nonzero elements (including explicitly presented zero values) in each column. + */ + def numNonzeros: Vector + + /** + * Maximum value of each column. + */ + def max: Vector + + /** + * Minimum value of each column. + */ + def min: Vector +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index ac2360c429e2b..901c3180eac4c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -17,14 +17,13 @@ package org.apache.spark.mllib.util -import breeze.linalg.{Vector => BV, DenseVector => BDV, SparseVector => BSV, - squaredDistance => breezeSquaredDistance} +import breeze.linalg.{Vector => BV, SparseVector => BSV, squaredDistance => breezeSquaredDistance} import org.apache.spark.annotation.Experimental import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.Vectors /** * Helper methods to load, save and pre-process data used in ML Lib. @@ -158,58 +157,6 @@ object MLUtils { dataStr.saveAsTextFile(dir) } - /** - * Utility function to compute mean and standard deviation on a given dataset. - * - * @param data - input data set whose statistics are computed - * @param numFeatures - number of features - * @param numExamples - number of examples in input dataset - * - * @return (yMean, xColMean, xColSd) - Tuple consisting of - * yMean - mean of the labels - * xColMean - Row vector with mean for every column (or feature) of the input data - * xColSd - Row vector standard deviation for every column (or feature) of the input data. - */ - private[mllib] def computeStats( - data: RDD[LabeledPoint], - numFeatures: Int, - numExamples: Long): (Double, Vector, Vector) = { - val brzData = data.map { case LabeledPoint(label, features) => - (label, features.toBreeze) - } - val aggStats = brzData.aggregate( - (0L, 0.0, BDV.zeros[Double](numFeatures), BDV.zeros[Double](numFeatures)) - )( - seqOp = (c, v) => (c, v) match { - case ((n, sumLabel, sum, sumSq), (label, features)) => - features.activeIterator.foreach { case (i, x) => - sumSq(i) += x * x - } - (n + 1L, sumLabel + label, sum += features, sumSq) - }, - combOp = (c1, c2) => (c1, c2) match { - case ((n1, sumLabel1, sum1, sumSq1), (n2, sumLabel2, sum2, sumSq2)) => - (n1 + n2, sumLabel1 + sumLabel2, sum1 += sum2, sumSq1 += sumSq2) - } - ) - val (nl, sumLabel, sum, sumSq) = aggStats - - require(nl > 0, "Input data is empty.") - require(nl == numExamples) - - val n = nl.toDouble - val yMean = sumLabel / n - val mean = sum / n - val std = new Array[Double](sum.length) - var i = 0 - while (i < numFeatures) { - std(i) = sumSq(i) / n - mean(i) * mean(i) - i += 1 - } - - (yMean, Vectors.fromBreeze(mean), Vectors.dense(std)) - } - /** * Returns the squared Euclidean distance between two vectors. The following formula will be used * if it does not introduce too much numerical error: diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index 71ee8e8a4f6fd..c9f9acf4c1335 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -170,4 +170,19 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { )) } } + + test("compute column summary statistics") { + for (mat <- Seq(denseMat, sparseMat)) { + val summary = mat.computeColumnSummaryStatistics() + // Run twice to make sure no internal states are changed. + for (k <- 0 to 1) { + assert(summary.mean === Vectors.dense(4.5, 3.0, 4.0), "mean mismatch") + assert(summary.variance === Vectors.dense(15.0, 10.0, 10.0), "variance mismatch") + assert(summary.count === m, "count mismatch.") + assert(summary.numNonzeros === Vectors.dense(3.0, 3.0, 4.0), "nnz mismatch") + assert(summary.max === Vectors.dense(9.0, 7.0, 8.0), "max mismatch") + assert(summary.min === Vectors.dense(0.0, 0.0, 1.0), "column mismatch.") + } + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index e451c350b8d88..812a8434784be 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -27,7 +27,6 @@ import com.google.common.base.Charsets import com.google.common.io.Files import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils._ class MLUtilsSuite extends FunSuite with LocalSparkContext { @@ -56,18 +55,6 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { } } - test("compute stats") { - val data = Seq.fill(3)(Seq( - LabeledPoint(1.0, Vectors.dense(1.0, 2.0, 3.0)), - LabeledPoint(0.0, Vectors.dense(3.0, 4.0, 5.0)) - )).flatten - val rdd = sc.parallelize(data, 2) - val (meanLabel, mean, std) = MLUtils.computeStats(rdd, 3, 6) - assert(meanLabel === 0.5) - assert(mean === Vectors.dense(2.0, 3.0, 4.0)) - assert(std === Vectors.dense(1.0, 1.0, 1.0)) - } - test("loadLibSVMData") { val lines = """ From aa8bb117a3ff98420ab751ba4ddbaad88ab57f9d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?baishuo=28=E7=99=BD=E7=A1=95=29?= Date: Fri, 11 Apr 2014 20:33:42 -0700 Subject: [PATCH 060/641] Update WindowedDStream.scala MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit update the content of Exception when windowDuration is not multiple of parent.slideDuration Author: baishuo(白硕) Closes #390 from baishuo/windowdstream and squashes the following commits: 533c968 [baishuo(白硕)] Update WindowedDStream.scala --- .../org/apache/spark/streaming/dstream/WindowedDStream.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala index 24289b714f99e..775b6bfd065c0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala @@ -32,7 +32,7 @@ class WindowedDStream[T: ClassTag]( extends DStream[T](parent.ssc) { if (!_windowDuration.isMultipleOf(parent.slideDuration)) { - throw new Exception("The window duration of windowed DStream (" + _slideDuration + ") " + + throw new Exception("The window duration of windowed DStream (" + _windowDuration + ") " + "must be a multiple of the slide duration of parent DStream (" + parent.slideDuration + ")") } From 165e06a74c3d75e6b7341c120943add8b035b96a Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 11 Apr 2014 22:46:47 -0700 Subject: [PATCH 061/641] SPARK-1057 (alternative) Remove fastutil (This is for discussion at this point -- I'm not suggesting this should be committed.) This is what removing fastutil looks like. Much of it is straightforward, like using `java.io` buffered stream classes, and Guava for murmurhash3. Uses of the `FastByteArrayOutputStream` were a little trickier. In only one case though do I think the change to use `java.io` actually entails an extra array copy. The rest is using `OpenHashMap` and `OpenHashSet`. These are now written in terms of more scala-like operations. `OpenHashMap` is where I made three non-trivial changes to make it work, and they need review: - It is no longer private - The key must be a `ClassTag` - Unless a lot of other code changes, the key type can't enforce being a supertype of `Null` It all works and tests pass, and I think there is reason to believe it's OK from a speed perspective. But what about those last changes? Author: Sean Owen Closes #266 from srowen/SPARK-1057-alternate and squashes the following commits: 2601129 [Sean Owen] Fix Map return type error not previously caught ec65502 [Sean Owen] Updates from matei's review 00bc81e [Sean Owen] Remove use of fastutil and replace with use of java.io, spark.util and Guava classes --- core/pom.xml | 4 --- .../spark/broadcast/HttpBroadcast.scala | 9 +++-- .../spark/partial/GroupedCountEvaluator.scala | 32 +++++++---------- .../main/scala/org/apache/spark/rdd/RDD.scala | 34 +++++++++---------- .../spark/scheduler/ReplayListenerBus.scala | 5 ++- .../org/apache/spark/scheduler/Task.scala | 9 ++--- .../apache/spark/serializer/Serializer.scala | 9 ++--- .../apache/spark/storage/BlockManager.scala | 10 +++--- .../spark/storage/BlockObjectWriter.scala | 6 ++-- .../org/apache/spark/util/FileLogger.scala | 5 ++- .../org/apache/spark/util/SizeEstimator.scala | 5 ++- .../spark/util/collection/AppendOnlyMap.scala | 7 ++-- .../collection/ExternalAppendOnlyMap.scala | 5 ++- .../spark/util/collection/OpenHashMap.scala | 3 +- .../spark/util/collection/OpenHashSet.scala | 4 +-- pom.xml | 5 --- project/SparkBuild.scala | 1 - .../spark/streaming/util/RawTextHelper.scala | 15 ++++---- .../spark/streaming/util/RawTextSender.scala | 11 +++--- 19 files changed, 72 insertions(+), 107 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 1f808380817c9..a1bdd8ec68aeb 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -157,10 +157,6 @@ - - it.unimi.dsi - fastutil - colt colt diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index f6a8a8af91e4b..29372f16f2cac 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -18,11 +18,10 @@ package org.apache.spark.broadcast import java.io.{File, FileOutputStream, ObjectInputStream, ObjectOutputStream, OutputStream} -import java.net.{URI, URL, URLConnection} +import java.io.{BufferedInputStream, BufferedOutputStream} +import java.net.{URL, URLConnection, URI} import java.util.concurrent.TimeUnit -import it.unimi.dsi.fastutil.io.{FastBufferedInputStream, FastBufferedOutputStream} - import org.apache.spark.{HttpServer, Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} @@ -164,7 +163,7 @@ private[spark] object HttpBroadcast extends Logging { if (compress) { compressionCodec.compressedOutputStream(new FileOutputStream(file)) } else { - new FastBufferedOutputStream(new FileOutputStream(file), bufferSize) + new BufferedOutputStream(new FileOutputStream(file), bufferSize) } } val ser = SparkEnv.get.serializer.newInstance() @@ -195,7 +194,7 @@ private[spark] object HttpBroadcast extends Logging { if (compress) { compressionCodec.compressedInputStream(inputStream) } else { - new FastBufferedInputStream(inputStream, bufferSize) + new BufferedInputStream(inputStream, bufferSize) } } val ser = SparkEnv.get.serializer.newInstance() diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala index 40b70baabcad9..8bb78123e3c9c 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala @@ -22,36 +22,33 @@ import java.util.{HashMap => JHashMap} import scala.collection.JavaConversions.mapAsScalaMap import scala.collection.Map import scala.collection.mutable.HashMap +import scala.reflect.ClassTag import cern.jet.stat.Probability -import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} + +import org.apache.spark.util.collection.OpenHashMap /** * An ApproximateEvaluator for counts by key. Returns a map of key to confidence interval. */ -private[spark] class GroupedCountEvaluator[T](totalOutputs: Int, confidence: Double) - extends ApproximateEvaluator[OLMap[T], Map[T, BoundedDouble]] { +private[spark] class GroupedCountEvaluator[T : ClassTag](totalOutputs: Int, confidence: Double) + extends ApproximateEvaluator[OpenHashMap[T,Long], Map[T, BoundedDouble]] { var outputsMerged = 0 - var sums = new OLMap[T] // Sum of counts for each key + var sums = new OpenHashMap[T,Long]() // Sum of counts for each key - override def merge(outputId: Int, taskResult: OLMap[T]) { + override def merge(outputId: Int, taskResult: OpenHashMap[T,Long]) { outputsMerged += 1 - val iter = taskResult.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - sums.put(entry.getKey, sums.getLong(entry.getKey) + entry.getLongValue) + taskResult.foreach { case (key, value) => + sums.changeValue(key, value, _ + value) } } override def currentResult(): Map[T, BoundedDouble] = { if (outputsMerged == totalOutputs) { val result = new JHashMap[T, BoundedDouble](sums.size) - val iter = sums.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - val sum = entry.getLongValue() - result(entry.getKey) = new BoundedDouble(sum, 1.0, sum, sum) + sums.foreach { case (key, sum) => + result(key) = new BoundedDouble(sum, 1.0, sum, sum) } result } else if (outputsMerged == 0) { @@ -60,16 +57,13 @@ private[spark] class GroupedCountEvaluator[T](totalOutputs: Int, confidence: Dou val p = outputsMerged.toDouble / totalOutputs val confFactor = Probability.normalInverse(1 - (1 - confidence) / 2) val result = new JHashMap[T, BoundedDouble](sums.size) - val iter = sums.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - val sum = entry.getLongValue + sums.foreach { case (key, sum) => val mean = (sum + 1 - p) / p val variance = (sum + 1) * (1 - p) / (p * p) val stdev = math.sqrt(variance) val low = mean - confFactor * stdev val high = mean + confFactor * stdev - result(entry.getKey) = new BoundedDouble(mean, confidence, low, high) + result(key) = new BoundedDouble(mean, confidence, low, high) } result } 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 3437b2cac19c2..891efccf23b6a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -20,12 +20,10 @@ package org.apache.spark.rdd import java.util.Random import scala.collection.Map -import scala.collection.JavaConversions.mapAsScalaMap import scala.collection.mutable.ArrayBuffer import scala.reflect.{classTag, ClassTag} import com.clearspring.analytics.stream.cardinality.HyperLogLog -import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} import org.apache.hadoop.io.BytesWritable import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.io.NullWritable @@ -43,6 +41,7 @@ import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel import org.apache.spark.util.{BoundedPriorityQueue, SerializableHyperLogLog, Utils} +import org.apache.spark.util.collection.OpenHashMap import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler} /** @@ -834,24 +833,24 @@ abstract class RDD[T: ClassTag]( throw new SparkException("countByValue() does not support arrays") } // TODO: This should perhaps be distributed by default. - def countPartition(iter: Iterator[T]): Iterator[OLMap[T]] = { - val map = new OLMap[T] - while (iter.hasNext) { - val v = iter.next() - map.put(v, map.getLong(v) + 1L) + def countPartition(iter: Iterator[T]): Iterator[OpenHashMap[T,Long]] = { + val map = new OpenHashMap[T,Long] + iter.foreach { + t => map.changeValue(t, 1L, _ + 1L) } Iterator(map) } - def mergeMaps(m1: OLMap[T], m2: OLMap[T]): OLMap[T] = { - val iter = m2.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - m1.put(entry.getKey, m1.getLong(entry.getKey) + entry.getLongValue) + def mergeMaps(m1: OpenHashMap[T,Long], m2: OpenHashMap[T,Long]): OpenHashMap[T,Long] = { + m2.foreach { case (key, value) => + m1.changeValue(key, value, _ + value) } m1 } val myResult = mapPartitions(countPartition).reduce(mergeMaps) - myResult.asInstanceOf[java.util.Map[T, Long]] // Will be wrapped as a Scala mutable Map + // Convert to a Scala mutable map + val mutableResult = scala.collection.mutable.Map[T,Long]() + myResult.foreach { case (k, v) => mutableResult.put(k, v) } + mutableResult } /** @@ -866,11 +865,10 @@ abstract class RDD[T: ClassTag]( if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValueApprox() does not support arrays") } - val countPartition: (TaskContext, Iterator[T]) => OLMap[T] = { (ctx, iter) => - val map = new OLMap[T] - while (iter.hasNext) { - val v = iter.next() - map.put(v, map.getLong(v) + 1L) + val countPartition: (TaskContext, Iterator[T]) => OpenHashMap[T,Long] = { (ctx, iter) => + val map = new OpenHashMap[T,Long] + iter.foreach { + t => map.changeValue(t, 1L, _ + 1L) } map } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index b03665fd56d33..f868e772cf58a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -17,11 +17,10 @@ package org.apache.spark.scheduler -import java.io.InputStream +import java.io.{BufferedInputStream, InputStream} import scala.io.Source -import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.hadoop.fs.{Path, FileSystem} import org.json4s.jackson.JsonMethods._ @@ -62,7 +61,7 @@ private[spark] class ReplayListenerBus( var currentLine = "" try { fileStream = Some(fileSystem.open(path)) - bufferedStream = Some(new FastBufferedInputStream(fileStream.get)) + bufferedStream = Some(new BufferedInputStream(fileStream.get)) compressStream = Some(wrapForCompression(bufferedStream.get)) // Parse each line as an event and post the event to all attached listeners diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index b85b4a50cd93a..a8bcb7dfe2f3c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -17,13 +17,11 @@ package org.apache.spark.scheduler -import java.io.{DataInputStream, DataOutputStream} +import java.io.{ByteArrayOutputStream, DataInputStream, DataOutputStream} import java.nio.ByteBuffer import scala.collection.mutable.HashMap -import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream - import org.apache.spark.TaskContext import org.apache.spark.executor.TaskMetrics import org.apache.spark.serializer.SerializerInstance @@ -104,7 +102,7 @@ private[spark] object Task { serializer: SerializerInstance) : ByteBuffer = { - val out = new FastByteArrayOutputStream(4096) + val out = new ByteArrayOutputStream(4096) val dataOut = new DataOutputStream(out) // Write currentFiles @@ -125,8 +123,7 @@ private[spark] object Task { dataOut.flush() val taskBytes = serializer.serialize(task).array() out.write(taskBytes) - out.trim() - ByteBuffer.wrap(out.array) + ByteBuffer.wrap(out.toByteArray) } /** 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 9f04dc6e427c0..f2c8f9b6218d6 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -17,11 +17,9 @@ package org.apache.spark.serializer -import java.io.{EOFException, InputStream, OutputStream} +import java.io.{ByteArrayOutputStream, EOFException, InputStream, OutputStream} import java.nio.ByteBuffer -import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream - import org.apache.spark.SparkEnv import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.{ByteBufferInputStream, NextIterator} @@ -73,10 +71,9 @@ trait SerializerInstance { def serializeMany[T](iterator: Iterator[T]): ByteBuffer = { // Default implementation uses serializeStream - val stream = new FastByteArrayOutputStream() + val stream = new ByteArrayOutputStream() serializeStream(stream).writeAll(iterator) - val buffer = ByteBuffer.allocate(stream.position.toInt) - buffer.put(stream.array, 0, stream.position.toInt) + val buffer = ByteBuffer.wrap(stream.toByteArray) buffer.flip() buffer } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index df9bb4044e37a..f14017051fa07 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import java.io.{File, InputStream, OutputStream} +import java.io.{File, InputStream, OutputStream, BufferedOutputStream, ByteArrayOutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -26,7 +26,6 @@ import scala.concurrent.duration._ import scala.util.Random import akka.actor.{ActorSystem, Cancellable, Props} -import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} import sun.nio.ch.DirectBuffer import org.apache.spark.{Logging, MapOutputTracker, SecurityManager, SparkConf, SparkEnv, SparkException} @@ -992,7 +991,7 @@ private[spark] class BlockManager( outputStream: OutputStream, values: Iterator[Any], serializer: Serializer = defaultSerializer) { - val byteStream = new FastBufferedOutputStream(outputStream) + val byteStream = new BufferedOutputStream(outputStream) val ser = serializer.newInstance() ser.serializeStream(wrapForCompression(blockId, byteStream)).writeAll(values).close() } @@ -1002,10 +1001,9 @@ private[spark] class BlockManager( blockId: BlockId, values: Iterator[Any], serializer: Serializer = defaultSerializer): ByteBuffer = { - val byteStream = new FastByteArrayOutputStream(4096) + val byteStream = new ByteArrayOutputStream(4096) dataSerializeStream(blockId, byteStream, values, serializer) - byteStream.trim() - ByteBuffer.wrap(byteStream.array) + ByteBuffer.wrap(byteStream.toByteArray) } /** diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index 696b930a26b9e..a2687e6be4e34 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -17,11 +17,9 @@ package org.apache.spark.storage -import java.io.{FileOutputStream, File, OutputStream} +import java.io.{BufferedOutputStream, FileOutputStream, File, OutputStream} import java.nio.channels.FileChannel -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream - import org.apache.spark.Logging import org.apache.spark.serializer.{SerializationStream, Serializer} @@ -119,7 +117,7 @@ private[spark] class DiskBlockObjectWriter( ts = new TimeTrackingOutputStream(fos) channel = fos.getChannel() lastValidPosition = initialPosition - bs = compressStream(new FastBufferedOutputStream(ts, bufferSize)) + bs = compressStream(new BufferedOutputStream(ts, bufferSize)) objOut = serializer.newInstance().serializeStream(bs) initialized = true this diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 0080a8b342b05..68a12e8ed67d7 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -17,12 +17,11 @@ package org.apache.spark.util -import java.io._ +import java.io.{FileOutputStream, BufferedOutputStream, PrintWriter, IOException} import java.net.URI import java.text.SimpleDateFormat import java.util.Date -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream import org.apache.hadoop.fs.{FSDataOutputStream, Path} import org.apache.spark.{Logging, SparkConf} @@ -100,7 +99,7 @@ private[spark] class FileLogger( hadoopDataStream.get } - val bstream = new FastBufferedOutputStream(dstream, outputBufferSize) + val bstream = new BufferedOutputStream(dstream, outputBufferSize) val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream new PrintWriter(cstream) } diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index b955612ca7749..08465575309c6 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -27,9 +27,8 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable.ArrayBuffer -import it.unimi.dsi.fastutil.ints.IntOpenHashSet - import org.apache.spark.Logging +import org.apache.spark.util.collection.OpenHashSet /** * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in @@ -207,7 +206,7 @@ private[spark] object SizeEstimator extends Logging { // Estimate the size of a large array by sampling elements without replacement. var size = 0.0 val rand = new Random(42) - val drawn = new IntOpenHashSet(ARRAY_SAMPLE_SIZE) + val drawn = new OpenHashSet[Int](ARRAY_SAMPLE_SIZE) for (i <- 0 until ARRAY_SAMPLE_SIZE) { var index = 0 do { diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index 025492b177a77..ad38250ad339f 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -19,6 +19,8 @@ package org.apache.spark.util.collection import java.util.{Arrays, Comparator} +import com.google.common.hash.Hashing + import org.apache.spark.annotation.DeveloperApi /** @@ -199,11 +201,8 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) /** * Re-hash a value to deal better with hash functions that don't differ in the lower bits. - * We use the Murmur Hash 3 finalization step that's also used in fastutil. */ - private def rehash(h: Int): Int = { - it.unimi.dsi.fastutil.HashCommon.murmurHash3(h) - } + private def rehash(h: Int): Int = Hashing.murmur3_32().hashInt(h).asInt() /** Double the table's size and re-hash everything */ protected def growTable() { 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 dd01ae821f705..d615767284c0b 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 @@ -17,14 +17,13 @@ package org.apache.spark.util.collection -import java.io._ +import java.io.{InputStream, BufferedInputStream, FileInputStream, File, Serializable, EOFException} import java.util.Comparator import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import com.google.common.io.ByteStreams -import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.annotation.DeveloperApi @@ -350,7 +349,7 @@ class ExternalAppendOnlyMap[K, V, C]( private class DiskMapIterator(file: File, blockId: BlockId, batchSizes: ArrayBuffer[Long]) extends Iterator[(K, C)] { private val fileStream = new FileInputStream(file) - private val bufferedStream = new FastBufferedInputStream(fileStream, fileBufferSize) + private val bufferedStream = new BufferedInputStream(fileStream, fileBufferSize) // An intermediate stream that reads from exactly one batch // This guards against pre-fetching and other arbitrary behavior of higher level streams diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index 62f99f3981793..b8de4ff9aa494 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -30,7 +30,8 @@ import org.apache.spark.annotation.DeveloperApi * Under the hood, it uses our OpenHashSet implementation. */ @DeveloperApi -class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( +private[spark] +class OpenHashMap[K : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( initialCapacity: Int) extends Iterable[(K, V)] with Serializable { diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index 148c12e64d2ce..19af4f8cbe428 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -18,6 +18,7 @@ package org.apache.spark.util.collection import scala.reflect._ +import com.google.common.hash.Hashing /** * A simple, fast hash set optimized for non-null insertion-only use case, where keys are never @@ -256,9 +257,8 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( /** * Re-hash a value to deal better with hash functions that don't differ in the lower bits. - * We use the Murmur Hash 3 finalization step that's also used in fastutil. */ - private def hashcode(h: Int): Int = it.unimi.dsi.fastutil.HashCommon.murmurHash3(h) + private def hashcode(h: Int): Int = Hashing.murmur3_32().hashInt(h).asInt() private def nextPowerOf2(n: Int): Int = { val highBit = Integer.highestOneBit(n) diff --git a/pom.xml b/pom.xml index c03bb35c99442..5f66cbe768592 100644 --- a/pom.xml +++ b/pom.xml @@ -348,11 +348,6 @@ - - it.unimi.dsi - fastutil - 6.4.4 - colt colt diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 21163760e6277..a6058bba3d211 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -331,7 +331,6 @@ object SparkBuild extends Build { "org.spark-project.akka" %% "akka-slf4j" % akkaVersion excludeAll(excludeNetty), "org.spark-project.akka" %% "akka-testkit" % akkaVersion % "test", "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), - "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.13.0", "commons-net" % "commons-net" % "2.2", diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala index bd1df55cf70f5..bbf57ef9275c0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala @@ -19,18 +19,17 @@ package org.apache.spark.streaming.util import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} +import org.apache.spark.util.collection.OpenHashMap import scala.collection.JavaConversions.mapAsScalaMap private[streaming] object RawTextHelper { - /** - * Splits lines and counts the words in them using specialized object-to-long hashmap - * (to avoid boxing-unboxing overhead of Long in java/scala HashMap) + /** + * Splits lines and counts the words. */ def splitAndCountPartitions(iter: Iterator[String]): Iterator[(String, Long)] = { - val map = new OLMap[String] + val map = new OpenHashMap[String,Long] var i = 0 var j = 0 while (iter.hasNext) { @@ -43,14 +42,16 @@ object RawTextHelper { } if (j > i) { val w = s.substring(i, j) - val c = map.getLong(w) - map.put(w, c + 1) + map.changeValue(w, 1L, _ + 1L) } i = j while (i < s.length && s.charAt(i) == ' ') { i += 1 } } + map.toIterator.map { + case (k, v) => (k, v) + } } map.toIterator.map{case (k, v) => (k, v)} } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala index 684b38e8b3102..a7850812bd612 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala @@ -17,14 +17,12 @@ package org.apache.spark.streaming.util -import java.io.IOException +import java.io.{ByteArrayOutputStream, IOException} import java.net.ServerSocket import java.nio.ByteBuffer import scala.io.Source -import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream - import org.apache.spark.{SparkConf, Logging} import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.IntParam @@ -45,16 +43,15 @@ object RawTextSender extends Logging { // Repeat the input data multiple times to fill in a buffer val lines = Source.fromFile(file).getLines().toArray - val bufferStream = new FastByteArrayOutputStream(blockSize + 1000) + val bufferStream = new ByteArrayOutputStream(blockSize + 1000) val ser = new KryoSerializer(new SparkConf()).newInstance() val serStream = ser.serializeStream(bufferStream) var i = 0 - while (bufferStream.position < blockSize) { + while (bufferStream.size < blockSize) { serStream.writeObject(lines(i)) i = (i + 1) % lines.length } - bufferStream.trim() - val array = bufferStream.array + val array = bufferStream.toByteArray val countBuf = ByteBuffer.wrap(new Array[Byte](4)) countBuf.putInt(array.length) From 6aa08c39cf30fa5c4ed97f4fff16371b9030a2e6 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 11 Apr 2014 23:33:49 -0700 Subject: [PATCH 062/641] [SPARK-1386] Web UI for Spark Streaming When debugging Spark Streaming applications it is necessary to monitor certain metrics that are not shown in the Spark application UI. For example, what is average processing time of batches? What is the scheduling delay? Is the system able to process as fast as it is receiving data? How many records I am receiving through my receivers? While the StreamingListener interface introduced in the 0.9 provided some of this information, it could only be accessed programmatically. A UI that shows information specific to the streaming applications is necessary for easier debugging. This PR introduces such a UI. It shows various statistics related to the streaming application. Here is a screenshot of the UI running on my local machine. http://i.imgur.com/1ooDGhm.png This UI is integrated into the Spark UI running at 4040. Author: Tathagata Das Author: Andrew Or Closes #290 from tdas/streaming-web-ui and squashes the following commits: fc73ca5 [Tathagata Das] Merge pull request #9 from andrewor14/ui-refactor 642dd88 [Andrew Or] Merge SparkUISuite.scala into UISuite.scala eb30517 [Andrew Or] Merge github.com:apache/spark into ui-refactor f4f4cbe [Tathagata Das] More minor fixes. 34bb364 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui 252c566 [Tathagata Das] Merge pull request #8 from andrewor14/ui-refactor e038b4b [Tathagata Das] Addressed Patrick's comments. 125a054 [Andrew Or] Disable serving static resources with gzip 90feb8d [Andrew Or] Address Patrick's comments 89dae36 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui 72fe256 [Tathagata Das] Merge pull request #6 from andrewor14/ui-refactor 2fc09c8 [Tathagata Das] Added binary check exclusions aa396d4 [Andrew Or] Rename tabs and pages (No more IndexPage.scala) f8e1053 [Tathagata Das] Added Spark and Streaming UI unit tests. caa5e05 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui 585cd65 [Tathagata Das] Merge pull request #5 from andrewor14/ui-refactor 914b8ff [Tathagata Das] Moved utils functions to UIUtils. 548c98c [Andrew Or] Wide refactoring of WebUI, UITab, and UIPage (see commit message) 6de06b0 [Tathagata Das] Merge remote-tracking branch 'apache/master' into streaming-web-ui ee6543f [Tathagata Das] Minor changes based on Andrew's comments. fa760fe [Tathagata Das] Fixed long line. 1c0bcef [Tathagata Das] Refactored streaming UI into two files. 1af239b [Tathagata Das] Changed streaming UI to attach itself as a tab with the Spark UI. 827e81a [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui 168fe86 [Tathagata Das] Merge pull request #2 from andrewor14/ui-refactor 3e986f8 [Tathagata Das] Merge remote-tracking branch 'apache/master' into streaming-web-ui c78c92d [Andrew Or] Remove outdated comment 8f7323b [Andrew Or] End of file new lines, indentation, and imports (minor) 0d61ee8 [Andrew Or] Merge branch 'streaming-web-ui' of github.com:tdas/spark into ui-refactor 9a48fa1 [Andrew Or] Allow adding tabs to SparkUI dynamically + add example 61358e3 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-web-ui 53be2c5 [Tathagata Das] Minor style updates. ed25dfc [Andrew Or] Generalize SparkUI header to display tabs dynamically a37ad4f [Andrew Or] Comments, imports and formatting (minor) cd000b0 [Andrew Or] Merge github.com:apache/spark into ui-refactor 7d57444 [Andrew Or] Refactoring the UI interface to add flexibility aef4dd5 [Tathagata Das] Added Apache licenses. db27bad [Tathagata Das] Added last batch processing time to StreamingUI. 4d86e98 [Tathagata Das] Added basic stats to the StreamingUI and refactored the UI to a Page to make it easier to transition to using SparkUI later. 93f1c69 [Tathagata Das] Added network receiver information to the Streaming UI. 56cc7fb [Tathagata Das] First cut implementation of Streaming UI. --- .../scala/org/apache/spark/SparkContext.scala | 1 - .../spark/deploy/SparkUIContainer.scala | 50 ----- .../{IndexPage.scala => HistoryPage.scala} | 12 +- .../spark/deploy/history/HistoryServer.scala | 61 +++--- .../apache/spark/deploy/master/Master.scala | 8 +- .../deploy/master/ui/ApplicationPage.scala | 13 +- .../ui/{IndexPage.scala => MasterPage.scala} | 23 ++- .../spark/deploy/master/ui/MasterWebUI.scala | 54 ++---- .../apache/spark/deploy/worker/Worker.scala | 2 +- .../spark/deploy/worker/ui/LogPage.scala | 147 ++++++++++++++ .../ui/{IndexPage.scala => WorkerPage.scala} | 6 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 180 +++--------------- .../scheduler/ApplicationEventListener.scala | 4 +- .../apache/spark/storage/StorageUtils.scala | 16 +- .../org/apache/spark/ui/JettyUtils.scala | 1 + .../main/scala/org/apache/spark/ui/Page.scala | 22 --- .../scala/org/apache/spark/ui/SparkUI.scala | 108 ++++------- .../scala/org/apache/spark/ui/UIUtils.scala | 172 +++++++++++++---- .../scala/org/apache/spark/ui/WebUI.scala | 141 +++++++++++--- ...ironmentUI.scala => EnvironmentPage.scala} | 47 +---- .../apache/spark/ui/env/EnvironmentTab.scala | 50 +++++ ...{ExecutorsUI.scala => ExecutorsPage.scala} | 84 +------- .../apache/spark/ui/exec/ExecutorsTab.scala | 86 +++++++++ .../apache/spark/ui/jobs/ExecutorTable.scala | 7 +- .../spark/ui/jobs/JobProgressListener.scala | 10 +- ...{IndexPage.scala => JobProgressPage.scala} | 16 +- ...bProgressUI.scala => JobProgressTab.scala} | 45 ++--- .../org/apache/spark/ui/jobs/PoolPage.scala | 14 +- .../org/apache/spark/ui/jobs/PoolTable.scala | 7 +- .../org/apache/spark/ui/jobs/StagePage.scala | 45 ++--- .../org/apache/spark/ui/jobs/StageTable.scala | 18 +- .../org/apache/spark/ui/storage/RDDPage.scala | 17 +- .../{IndexPage.scala => StoragePage.scala} | 13 +- ...{BlockManagerUI.scala => StorageTab.scala} | 32 +--- .../org/apache/spark/util/JsonProtocol.scala | 12 +- .../scala/org/apache/spark/ui/UISuite.scala | 81 +++++++- .../apache/spark/util/JsonProtocolSuite.scala | 4 +- project/MimaBuild.scala | 8 +- .../spark/streaming/StreamingContext.scala | 23 +-- .../spark/streaming/dstream/DStream.scala | 9 - .../dstream/NetworkInputDStream.scala | 79 +++++--- .../spark/streaming/scheduler/BatchInfo.scala | 1 + .../streaming/scheduler/JobGenerator.scala | 9 +- .../streaming/scheduler/JobScheduler.scala | 11 +- .../spark/streaming/scheduler/JobSet.scala | 7 +- .../scheduler/NetworkInputTracker.scala | 86 ++++++--- .../scheduler/StreamingListener.scala | 18 +- .../scheduler/StreamingListenerBus.scala | 4 + .../ui/StreamingJobProgressListener.scala | 148 ++++++++++++++ .../spark/streaming/ui/StreamingPage.scala | 180 ++++++++++++++++++ .../spark/streaming/ui/StreamingTab.scala | 27 +-- .../spark/streaming/InputStreamsSuite.scala | 6 +- .../streaming/StreamingContextSuite.scala | 1 - .../org/apache/spark/streaming/UISuite.scala | 46 +++++ 54 files changed, 1426 insertions(+), 846 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala rename core/src/main/scala/org/apache/spark/deploy/history/{IndexPage.scala => HistoryPage.scala} (85%) rename core/src/main/scala/org/apache/spark/deploy/master/ui/{IndexPage.scala => MasterPage.scala} (91%) create mode 100644 core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala rename core/src/main/scala/org/apache/spark/deploy/worker/ui/{IndexPage.scala => WorkerPage.scala} (97%) delete mode 100644 core/src/main/scala/org/apache/spark/ui/Page.scala rename core/src/main/scala/org/apache/spark/ui/env/{EnvironmentUI.scala => EnvironmentPage.scala} (61%) create mode 100644 core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala rename core/src/main/scala/org/apache/spark/ui/exec/{ExecutorsUI.scala => ExecutorsPage.scala} (61%) create mode 100644 core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala rename core/src/main/scala/org/apache/spark/ui/jobs/{IndexPage.scala => JobProgressPage.scala} (90%) rename core/src/main/scala/org/apache/spark/ui/jobs/{JobProgressUI.scala => JobProgressTab.scala} (53%) rename core/src/main/scala/org/apache/spark/ui/storage/{IndexPage.scala => StoragePage.scala} (90%) rename core/src/main/scala/org/apache/spark/ui/storage/{BlockManagerUI.scala => StorageTab.scala} (75%) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala rename core/src/test/scala/org/apache/spark/SparkUISuite.scala => streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala (58%) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3bcc8ce2b25a6..a764c174d562c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -213,7 +213,6 @@ class SparkContext(config: SparkConf) extends Logging { // Initialize the Spark UI, registering all associated listeners private[spark] val ui = new SparkUI(this) ui.bind() - ui.start() // Optionally log Spark events private[spark] val eventLogger: Option[EventLoggingListener] = { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala b/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala deleted file mode 100644 index 33fceae4ff489..0000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala +++ /dev/null @@ -1,50 +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.deploy - -import org.apache.spark.ui.{SparkUI, WebUI} - -private[spark] abstract class SparkUIContainer(name: String) extends WebUI(name) { - - /** Attach a SparkUI to this container. Only valid after bind(). */ - def attachUI(ui: SparkUI) { - assert(serverInfo.isDefined, - "%s must be bound to a server before attaching SparkUIs".format(name)) - val rootHandler = serverInfo.get.rootHandler - for (handler <- ui.handlers) { - rootHandler.addHandler(handler) - if (!handler.isStarted) { - handler.start() - } - } - } - - /** Detach a SparkUI from this container. Only valid after bind(). */ - def detachUI(ui: SparkUI) { - assert(serverInfo.isDefined, - "%s must be bound to a server before detaching SparkUIs".format(name)) - val rootHandler = serverInfo.get.rootHandler - for (handler <- ui.handlers) { - if (handler.isStarted) { - handler.stop() - } - rootHandler.removeHandler(handler) - } - } - -} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala similarity index 85% rename from core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala rename to core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 54dffffec71c5..180c853ce3096 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -21,9 +21,9 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.{UIUtils, WebUI} +import org.apache.spark.ui.{WebUIPage, UIUtils} -private[spark] class IndexPage(parent: HistoryServer) { +private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated } @@ -62,13 +62,13 @@ private[spark] class IndexPage(parent: HistoryServer) { private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { val appName = if (info.started) info.name else info.logDirPath.getName val uiAddress = parent.getAddress + info.ui.basePath - val startTime = if (info.started) WebUI.formatDate(info.startTime) else "Not started" - val endTime = if (info.completed) WebUI.formatDate(info.endTime) else "Not completed" + val startTime = if (info.started) UIUtils.formatDate(info.startTime) else "Not started" + val endTime = if (info.completed) UIUtils.formatDate(info.endTime) else "Not completed" val difference = if (info.started && info.completed) info.endTime - info.startTime else -1L - val duration = if (difference > 0) WebUI.formatDuration(difference) else "---" + val duration = if (difference > 0) UIUtils.formatDuration(difference) else "---" val sparkUser = if (info.started) info.sparkUser else "Unknown user" val logDirectory = info.logDirPath.getName - val lastUpdated = WebUI.formatDate(info.lastUpdated) + val lastUpdated = UIUtils.formatDate(info.lastUpdated) {appName} {startTime} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 97d2ba9deed33..cf64700f9098c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -17,17 +17,13 @@ package org.apache.spark.deploy.history -import javax.servlet.http.HttpServletRequest - import scala.collection.mutable import org.apache.hadoop.fs.{FileStatus, Path} -import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{Logging, SecurityManager, SparkConf} -import org.apache.spark.deploy.SparkUIContainer import org.apache.spark.scheduler._ -import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.{WebUI, SparkUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.Utils @@ -46,17 +42,15 @@ import org.apache.spark.util.Utils */ class HistoryServer( val baseLogDir: String, + securityManager: SecurityManager, conf: SparkConf) - extends SparkUIContainer("History Server") with Logging { + extends WebUI(securityManager, HistoryServer.WEB_UI_PORT, conf) with Logging { import HistoryServer._ private val fileSystem = Utils.getHadoopFileSystem(baseLogDir) private val localHost = Utils.localHostName() private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) - private val port = WEB_UI_PORT - private val securityManager = new SecurityManager(conf) - private val indexPage = new IndexPage(this) // A timestamp of when the disk was last accessed to check for log updates private var lastLogCheckTime = -1L @@ -90,37 +84,23 @@ class HistoryServer( } } - private val handlers = Seq[ServletContextHandler]( - createStaticHandler(STATIC_RESOURCE_DIR, "/static"), - createServletHandler("/", - (request: HttpServletRequest) => indexPage.render(request), securityMgr = securityManager) - ) - // A mapping of application ID to its history information, which includes the rendered UI val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]() + initialize() + /** - * Start the history server. + * Initialize the history server. * * This starts a background thread that periodically synchronizes information displayed on * this UI with the event logs in the provided base directory. */ - def start() { + def initialize() { + attachPage(new HistoryPage(this)) + attachHandler(createStaticHandler(STATIC_RESOURCE_DIR, "/static")) logCheckingThread.start() } - /** Bind to the HTTP server behind this web interface. */ - override def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf)) - logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort)) - } catch { - case e: Exception => - logError("Failed to bind HistoryServer", e) - System.exit(1) - } - } - /** * Check for any updates to event logs in the base directory. This is only effective once * the server has been bound. @@ -151,7 +131,7 @@ class HistoryServer( // Remove any applications that should no longer be retained appIdToInfo.foreach { case (appId, info) => if (!retainedAppIds.contains(appId)) { - detachUI(info.ui) + detachSparkUI(info.ui) appIdToInfo.remove(appId) } } @@ -186,15 +166,14 @@ class HistoryServer( val path = logDir.getPath val appId = path.getName val replayBus = new ReplayListenerBus(logInfo.logPaths, fileSystem, logInfo.compressionCodec) - val ui = new SparkUI(replayBus, appId, "/history/" + appId) val appListener = new ApplicationEventListener replayBus.addListener(appListener) + val ui = new SparkUI(conf, replayBus, appId, "/history/" + appId) // Do not call ui.bind() to avoid creating a new server for each application - ui.start() replayBus.replay() if (appListener.applicationStarted) { - attachUI(ui) + attachSparkUI(ui) val appName = appListener.appName val sparkUser = appListener.sparkUser val startTime = appListener.startTime @@ -213,6 +192,18 @@ class HistoryServer( fileSystem.close() } + /** Attach a reconstructed UI to this server. Only valid after bind(). */ + private def attachSparkUI(ui: SparkUI) { + assert(serverInfo.isDefined, "HistoryServer must be bound before attaching SparkUIs") + ui.getHandlers.foreach(attachHandler) + } + + /** Detach a reconstructed UI from this server. Only valid after bind(). */ + private def detachSparkUI(ui: SparkUI) { + assert(serverInfo.isDefined, "HistoryServer must be bound before detaching SparkUIs") + ui.getHandlers.foreach(detachHandler) + } + /** Return the address of this server. */ def getAddress: String = "http://" + publicHost + ":" + boundPort @@ -262,9 +253,9 @@ object HistoryServer { def main(argStrings: Array[String]) { val args = new HistoryServerArguments(argStrings) - val server = new HistoryServer(args.logDir, conf) + val securityManager = new SecurityManager(conf) + val server = new HistoryServer(args.logDir, securityManager, conf) server.bind() - server.start() // Wait until the end of the world... or if the HistoryServer process is manually stopped while(true) { Thread.sleep(Int.MaxValue) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 2446e86cb6672..6c58e741df001 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -625,7 +625,7 @@ private[spark] class Master( if (completedApps.size >= RETAINED_APPLICATIONS) { val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) completedApps.take(toRemove).foreach( a => { - appIdToUI.remove(a.id).foreach { ui => webUi.detachUI(ui) } + appIdToUI.remove(a.id).foreach { ui => webUi.detachSparkUI(ui) } applicationMetricsSystem.removeSource(a.appSource) }) completedApps.trimStart(toRemove) @@ -667,12 +667,12 @@ private[spark] class Master( if (!eventLogPaths.isEmpty) { try { val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) - val ui = new SparkUI(replayBus, appName + " (completed)", "/history/" + app.id) - ui.start() + val ui = new SparkUI( + new SparkConf, replayBus, appName + " (completed)", "/history/" + app.id) replayBus.replay() app.desc.appUiUrl = ui.basePath appIdToUI(app.id) = ui - webUi.attachUI(ui) + webUi.attachSparkUI(ui) return true } catch { case t: Throwable => diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index cb092cb5d576b..b5cd4d2ea963f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -28,15 +28,16 @@ import org.json4s.JValue import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.ExecutorInfo -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class ApplicationPage(parent: MasterWebUI) { - val master = parent.masterActorRef - val timeout = parent.timeout +private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") { + + private val master = parent.masterActorRef + private val timeout = parent.timeout /** Executor details for a particular application */ - def renderJson(request: HttpServletRequest): JValue = { + override def renderJson(request: HttpServletRequest): JValue = { val appId = request.getParameter("appId") val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, timeout) @@ -96,7 +97,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { UIUtils.basicSparkPage(content, "Application: " + app.desc.name) } - def executorRow(executor: ExecutorInfo): Seq[Node] = { + private def executorRow(executor: ExecutorInfo): Seq[Node] = { {executor.id} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala similarity index 91% rename from core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala rename to core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 8c1d6c7cce450..7ca3b08a28728 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -25,17 +25,17 @@ import scala.xml.Node import akka.pattern.ask import org.json4s.JValue -import org.apache.spark.deploy.{JsonProtocol} +import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} -import org.apache.spark.ui.{WebUI, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class IndexPage(parent: MasterWebUI) { - val master = parent.masterActorRef - val timeout = parent.timeout +private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { + private val master = parent.masterActorRef + private val timeout = parent.timeout - def renderJson(request: HttpServletRequest): JValue = { + override def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, timeout) JsonProtocol.writeMasterState(state) @@ -139,7 +139,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { UIUtils.basicSparkPage(content, "Spark Master at " + state.uri) } - def workerRow(worker: WorkerInfo): Seq[Node] = { + private def workerRow(worker: WorkerInfo): Seq[Node] = { {worker.id} @@ -154,8 +154,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { } - - def appRow(app: ApplicationInfo): Seq[Node] = { + private def appRow(app: ApplicationInfo): Seq[Node] = { {app.id} @@ -169,14 +168,14 @@ private[spark] class IndexPage(parent: MasterWebUI) { {Utils.megabytesToString(app.desc.memoryPerSlave)} - {WebUI.formatDate(app.submitDate)} + {UIUtils.formatDate(app.submitDate)} {app.desc.user} {app.state.toString} - {WebUI.formatDuration(app.duration)} + {UIUtils.formatDuration(app.duration)} } - def driverRow(driver: DriverInfo): Seq[Node] = { + private def driverRow(driver: DriverInfo): Seq[Node] = { {driver.id} {driver.submitDate} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 30c8ade408a5a..a18b39fc95d64 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -17,14 +17,9 @@ package org.apache.spark.deploy.master.ui -import javax.servlet.http.HttpServletRequest - -import org.eclipse.jetty.servlet.ServletContextHandler - import org.apache.spark.Logging -import org.apache.spark.deploy.SparkUIContainer import org.apache.spark.deploy.master.Master -import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -33,44 +28,33 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class MasterWebUI(val master: Master, requestedPort: Int) - extends SparkUIContainer("MasterWebUI") with Logging { + extends WebUI(master.securityMgr, requestedPort, master.conf) with Logging { val masterActorRef = master.self val timeout = AkkaUtils.askTimeout(master.conf) - private val host = Utils.localHostName() - private val port = requestedPort - private val applicationPage = new ApplicationPage(this) - private val indexPage = new IndexPage(this) + initialize() - private val handlers: Seq[ServletContextHandler] = { - master.masterMetricsSystem.getServletHandlers ++ - master.applicationMetricsSystem.getServletHandlers ++ - Seq[ServletContextHandler]( - createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static"), - createServletHandler("/app/json", - (request: HttpServletRequest) => applicationPage.renderJson(request), master.securityMgr), - createServletHandler("/app", - (request: HttpServletRequest) => applicationPage.render(request), master.securityMgr), - createServletHandler("/json", - (request: HttpServletRequest) => indexPage.renderJson(request), master.securityMgr), - createServletHandler("/", - (request: HttpServletRequest) => indexPage.render(request), master.securityMgr) - ) + /** Initialize all components of the server. */ + def initialize() { + attachPage(new ApplicationPage(this)) + attachPage(new MasterPage(this)) + attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) + master.masterMetricsSystem.getServletHandlers.foreach(attachHandler) + master.applicationMetricsSystem.getServletHandlers.foreach(attachHandler) } - /** Bind to the HTTP server behind this web interface. */ - override def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, master.conf)) - logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Master web UI", e) - System.exit(1) - } + /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ + def attachSparkUI(ui: SparkUI) { + assert(serverInfo.isDefined, "Master UI must be bound to a server before attaching SparkUIs") + ui.getHandlers.foreach(attachHandler) } + /** Detach a reconstructed UI from this Master UI. Only valid after bind(). */ + def detachSparkUI(ui: SparkUI) { + assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") + ui.getHandlers.foreach(detachHandler) + } } private[spark] object MasterWebUI { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index bf5a8d09dd2df..52c164ca3c574 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -128,8 +128,8 @@ private[spark] class Worker( host, port, cores, Utils.megabytesToString(memory))) logInfo("Spark home: " + sparkHome) createWorkDir() - webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) + webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) webUi.bind() registerWithMaster() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala new file mode 100644 index 0000000000000..fec1207948628 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -0,0 +1,147 @@ +/* + * 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.deploy.worker.ui + +import java.io.File +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.util.Utils + +private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { + private val worker = parent.worker + private val workDir = parent.workDir + + def renderLog(request: HttpServletRequest): String = { + val defaultBytes = 100 * 1024 + + val appId = Option(request.getParameter("appId")) + val executorId = Option(request.getParameter("executorId")) + val driverId = Option(request.getParameter("driverId")) + val logType = request.getParameter("logType") + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) + + val path = (appId, executorId, driverId) match { + case (Some(a), Some(e), None) => + s"${workDir.getPath}/$appId/$executorId/$logType" + case (None, None, Some(d)) => + s"${workDir.getPath}/$driverId/$logType" + case _ => + throw new Exception("Request must specify either application or driver identifiers") + } + + val (startByte, endByte) = getByteRange(path, offset, byteLength) + val file = new File(path) + val logLength = file.length + + val pre = s"==== Bytes $startByte-$endByte of $logLength of $path ====\n" + pre + Utils.offsetBytes(path, startByte, endByte) + } + + def render(request: HttpServletRequest): Seq[Node] = { + val defaultBytes = 100 * 1024 + val appId = Option(request.getParameter("appId")) + val executorId = Option(request.getParameter("executorId")) + val driverId = Option(request.getParameter("driverId")) + val logType = request.getParameter("logType") + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) + + val (path, params) = (appId, executorId, driverId) match { + case (Some(a), Some(e), None) => + (s"${workDir.getPath}/$a/$e/$logType", s"appId=$a&executorId=$e") + case (None, None, Some(d)) => + (s"${workDir.getPath}/$d/$logType", s"driverId=$d") + case _ => + throw new Exception("Request must specify either application or driver identifiers") + } + + val (startByte, endByte) = getByteRange(path, offset, byteLength) + val file = new File(path) + val logLength = file.length + val logText = {Utils.offsetBytes(path, startByte, endByte)} + val linkToMaster =

    Back to Master

    + val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} + + val backButton = + if (startByte > 0) { + + + + } + else { + + } + + val nextButton = + if (endByte < logLength) { + + + + } + else { + + } + + val content = + + + {linkToMaster} +
    +
    {backButton}
    +
    {range}
    +
    {nextButton}
    +
    +
    +
    +
    {logText}
    +
    + + + UIUtils.basicSparkPage(content, logType + " log page for " + appId) + } + + /** Determine the byte range for a log or log page. */ + private def getByteRange(path: String, offset: Option[Long], byteLength: Int): (Long, Long) = { + val defaultBytes = 100 * 1024 + val maxBytes = 1024 * 1024 + val file = new File(path) + val logLength = file.length() + val getOffset = offset.getOrElse(logLength - defaultBytes) + val startByte = + if (getOffset < 0) 0L + else if (getOffset > logLength) logLength + else getOffset + val logPageLength = math.min(byteLength, maxBytes) + val endByte = math.min(startByte + logPageLength, logLength) + (startByte, endByte) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala rename to core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala index 49c1009cac2bf..d4513118ced05 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala @@ -28,15 +28,15 @@ import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class IndexPage(parent: WorkerWebUI) { +private[spark] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { val workerActor = parent.worker.self val worker = parent.worker val timeout = parent.timeout - def renderJson(request: HttpServletRequest): JValue = { + override def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] val workerState = Await.result(stateFuture, timeout) JsonProtocol.writeWorkerState(workerState) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 5625a44549aaa..0ad2edba2227f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -20,174 +20,44 @@ package org.apache.spark.deploy.worker.ui import java.io.File import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.servlet.ServletContextHandler - -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.worker.Worker -import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} +import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.AkkaUtils /** * Web UI server for the standalone worker. */ private[spark] -class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) - extends WebUI("WorkerWebUI") with Logging { +class WorkerWebUI( + val worker: Worker, + val workDir: File, + port: Option[Int] = None) + extends WebUI(worker.securityMgr, WorkerWebUI.getUIPort(port, worker.conf), worker.conf) + with Logging { val timeout = AkkaUtils.askTimeout(worker.conf) - private val host = Utils.localHostName() - private val port = requestedPort.getOrElse( - worker.conf.getInt("worker.ui.port", WorkerWebUI.DEFAULT_PORT)) - private val indexPage = new IndexPage(this) - - private val handlers: Seq[ServletContextHandler] = { - worker.metricsSystem.getServletHandlers ++ - Seq[ServletContextHandler]( - createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static"), - createServletHandler("/log", - (request: HttpServletRequest) => log(request), worker.securityMgr), - createServletHandler("/logPage", - (request: HttpServletRequest) => logPage(request), worker.securityMgr), - createServletHandler("/json", - (request: HttpServletRequest) => indexPage.renderJson(request), worker.securityMgr), - createServletHandler("/", - (request: HttpServletRequest) => indexPage.render(request), worker.securityMgr) - ) - } - - /** Bind to the HTTP server behind this web interface. */ - override def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, worker.conf)) - logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Worker web UI", e) - System.exit(1) - } - } - - private def log(request: HttpServletRequest): String = { - val defaultBytes = 100 * 1024 - - val appId = Option(request.getParameter("appId")) - val executorId = Option(request.getParameter("executorId")) - val driverId = Option(request.getParameter("driverId")) - val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) - - val path = (appId, executorId, driverId) match { - case (Some(a), Some(e), None) => - s"${workDir.getPath}/$appId/$executorId/$logType" - case (None, None, Some(d)) => - s"${workDir.getPath}/$driverId/$logType" - case _ => - throw new Exception("Request must specify either application or driver identifiers") - } - - val (startByte, endByte) = getByteRange(path, offset, byteLength) - val file = new File(path) - val logLength = file.length - - val pre = s"==== Bytes $startByte-$endByte of $logLength of $path ====\n" - pre + Utils.offsetBytes(path, startByte, endByte) - } - - private def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { - val defaultBytes = 100 * 1024 - val appId = Option(request.getParameter("appId")) - val executorId = Option(request.getParameter("executorId")) - val driverId = Option(request.getParameter("driverId")) - val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) - - val (path, params) = (appId, executorId, driverId) match { - case (Some(a), Some(e), None) => - (s"${workDir.getPath}/$a/$e/$logType", s"appId=$a&executorId=$e") - case (None, None, Some(d)) => - (s"${workDir.getPath}/$d/$logType", s"driverId=$d") - case _ => - throw new Exception("Request must specify either application or driver identifiers") - } - - val (startByte, endByte) = getByteRange(path, offset, byteLength) - val file = new File(path) - val logLength = file.length - val logText = {Utils.offsetBytes(path, startByte, endByte)} - val linkToMaster =

    Back to Master

    - val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} - - val backButton = - if (startByte > 0) { - - - - } - else { - - } - - val nextButton = - if (endByte < logLength) { - - - - } - else { - - } - - val content = - - - {linkToMaster} -
    -
    {backButton}
    -
    {range}
    -
    {nextButton}
    -
    -
    -
    -
    {logText}
    -
    - - - UIUtils.basicSparkPage(content, logType + " log page for " + appId) + initialize() + + /** Initialize all components of the server. */ + def initialize() { + val logPage = new LogPage(this) + attachPage(logPage) + attachPage(new WorkerPage(this)) + attachHandler(createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static")) + attachHandler(createServletHandler("/log", + (request: HttpServletRequest) => logPage.renderLog(request), worker.securityMgr)) + worker.metricsSystem.getServletHandlers.foreach(attachHandler) } - - /** Determine the byte range for a log or log page. */ - private def getByteRange(path: String, offset: Option[Long], byteLength: Int): (Long, Long) = { - val defaultBytes = 100 * 1024 - val maxBytes = 1024 * 1024 - val file = new File(path) - val logLength = file.length() - val getOffset = offset.getOrElse(logLength - defaultBytes) - val startByte = - if (getOffset < 0) 0L - else if (getOffset > logLength) logLength - else getOffset - val logPageLength = math.min(byteLength, maxBytes) - val endByte = math.min(startByte + logPageLength, logLength) - (startByte, endByte) - } - } private[spark] object WorkerWebUI { - val DEFAULT_PORT=8081 + val DEFAULT_PORT = 8081 val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR + + def getUIPort(requestedPort: Option[Int], conf: SparkConf): Int = { + requestedPort.getOrElse(conf.getInt("worker.ui.port", WorkerWebUI.DEFAULT_PORT)) + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index affda13df6531..c1001227151a5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -31,11 +31,11 @@ private[spark] class ApplicationEventListener extends SparkListener { def applicationStarted = startTime != -1 - def applicationFinished = endTime != -1 + def applicationCompleted = endTime != -1 def applicationDuration: Long = { val difference = endTime - startTime - if (applicationStarted && applicationFinished && difference > 0) difference else -1L + if (applicationStarted && applicationCompleted && difference > 0) difference else -1L } override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 07255aa366a6d..7ed371326855d 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -42,24 +42,22 @@ class StorageStatus( def memRemaining : Long = maxMem - memUsed() - def rddBlocks = blocks.flatMap { - case (rdd: RDDBlockId, status) => Some(rdd, status) - case _ => None - } + def rddBlocks = blocks.collect { case (rdd: RDDBlockId, status) => (rdd, status) } } @DeveloperApi private[spark] class RDDInfo( - val id: Int, - val name: String, - val numPartitions: Int, - val storageLevel: StorageLevel) extends Ordered[RDDInfo] { + val id: Int, + val name: String, + val numPartitions: Int, + val storageLevel: StorageLevel) + extends Ordered[RDDInfo] { var numCachedPartitions = 0 var memSize = 0L var diskSize = 0L - var tachyonSize= 0L + var tachyonSize = 0L override def toString = { import Utils.bytesToString 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 dd0818e8ab01c..62a4e3d0f6a42 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -121,6 +121,7 @@ private[spark] object JettyUtils extends Logging { /** Create a handler for serving files from a static directory */ def createStaticHandler(resourceBase: String, path: String): ServletContextHandler = { val contextHandler = new ServletContextHandler + contextHandler.setInitParameter("org.eclipse.jetty.servlet.Default.gzip", "false") val staticHandler = new DefaultServlet val holder = new ServletHolder(staticHandler) Option(getClass.getClassLoader.getResource(resourceBase)) match { diff --git a/core/src/main/scala/org/apache/spark/ui/Page.scala b/core/src/main/scala/org/apache/spark/ui/Page.scala deleted file mode 100644 index b2a069a37552d..0000000000000 --- a/core/src/main/scala/org/apache/spark/ui/Page.scala +++ /dev/null @@ -1,22 +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.ui - -private[spark] object Page extends Enumeration { - val Stages, Storage, Environment, Executors = Value -} diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 7fa4fd3149eb6..2fef1a635427c 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,112 +17,86 @@ package org.apache.spark.ui -import org.eclipse.jetty.servlet.ServletContextHandler - -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkEnv} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.env.EnvironmentUI -import org.apache.spark.ui.exec.ExecutorsUI -import org.apache.spark.ui.jobs.JobProgressUI -import org.apache.spark.ui.storage.BlockManagerUI -import org.apache.spark.util.Utils +import org.apache.spark.ui.env.EnvironmentTab +import org.apache.spark.ui.exec.ExecutorsTab +import org.apache.spark.ui.jobs.JobProgressTab +import org.apache.spark.ui.storage.StorageTab -/** Top level user interface for Spark */ +/** + * Top level user interface for a Spark application. + */ private[spark] class SparkUI( val sc: SparkContext, val conf: SparkConf, + val securityManager: SecurityManager, val listenerBus: SparkListenerBus, var appName: String, val basePath: String = "") - extends WebUI("SparkUI") with Logging { + extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath) + with Logging { - def this(sc: SparkContext) = this(sc, sc.conf, sc.listenerBus, sc.appName) - def this(listenerBus: SparkListenerBus, appName: String, basePath: String) = - this(null, new SparkConf, listenerBus, appName, basePath) + def this(sc: SparkContext) = this(sc, sc.conf, sc.env.securityManager, sc.listenerBus, sc.appName) + def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = + this(null, conf, new SecurityManager(conf), listenerBus, appName, basePath) // If SparkContext is not provided, assume the associated application is not live val live = sc != null - val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) - - private val localHost = Utils.localHostName() - private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) - private val port = conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) + // Maintain executor storage status through Spark events + val storageStatusListener = new StorageStatusListener - private val storage = new BlockManagerUI(this) - private val jobs = new JobProgressUI(this) - private val env = new EnvironmentUI(this) - private val exec = new ExecutorsUI(this) + initialize() - val handlers: Seq[ServletContextHandler] = { - val metricsServletHandlers = if (live) { - SparkEnv.get.metricsSystem.getServletHandlers - } else { - Array[ServletContextHandler]() + /** Initialize all components of the server. */ + def initialize() { + listenerBus.addListener(storageStatusListener) + val jobProgressTab = new JobProgressTab(this) + attachTab(jobProgressTab) + attachTab(new StorageTab(this)) + attachTab(new EnvironmentTab(this)) + attachTab(new ExecutorsTab(this)) + attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) + attachHandler(createRedirectHandler("/", "/stages", basePath = basePath)) + attachHandler( + createRedirectHandler("/stages/stage/kill", "/stages", jobProgressTab.handleKillRequest)) + if (live) { + sc.env.metricsSystem.getServletHandlers.foreach(attachHandler) } - storage.getHandlers ++ - jobs.getHandlers ++ - env.getHandlers ++ - exec.getHandlers ++ - metricsServletHandlers ++ - Seq[ServletContextHandler] ( - createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"), - createRedirectHandler("/", "/stages", basePath = basePath) - ) } - // Maintain executor storage status through Spark events - val storageStatusListener = new StorageStatusListener - + /** Set the app name for this UI. */ def setAppName(name: String) { appName = name } - /** Initialize all components of the server */ - def start() { - storage.start() - jobs.start() - env.start() - exec.start() - - // Storage status listener must receive events first, as other listeners depend on its state - listenerBus.addListener(storageStatusListener) - listenerBus.addListener(storage.listener) - listenerBus.addListener(jobs.listener) - listenerBus.addListener(env.listener) - listenerBus.addListener(exec.listener) - } - - /** Bind to the HTTP server behind this web interface. */ - override def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, sc.conf)) - logInfo("Started Spark web UI at http://%s:%d".format(publicHost, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Spark web UI", e) - System.exit(1) - } + /** Register the given listener with the listener bus. */ + def registerListener(listener: SparkListener) { + listenerBus.addListener(listener) } /** Stop the server behind this web interface. Only valid after bind(). */ override def stop() { super.stop() - logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) + logInfo("Stopped Spark web UI at %s".format(appUIAddress)) } /** * Return the application UI host:port. This does not include the scheme (http://). */ - private[spark] def appUIHostPort = publicHost + ":" + boundPort + private[spark] def appUIHostPort = publicHostName + ":" + boundPort private[spark] def appUIAddress = s"http://$appUIHostPort" - } private[spark] object SparkUI { val DEFAULT_PORT = 4040 val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" + + def getUIPort(conf: SparkConf): Int = { + conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) + } } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index a7cf04b3cbb86..6a2d652528d8a 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -17,16 +17,115 @@ package org.apache.spark.ui +import java.text.SimpleDateFormat +import java.util.{Locale, Date} + import scala.xml.Node +import org.apache.spark.Logging /** Utility functions for generating XML pages with spark content. */ -private[spark] object UIUtils { +private[spark] object UIUtils extends Logging { + + // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. + private val dateFormat = new ThreadLocal[SimpleDateFormat]() { + override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + } + + def formatDate(date: Date): String = dateFormat.get.format(date) + + def formatDate(timestamp: Long): String = dateFormat.get.format(new Date(timestamp)) + + def formatDuration(milliseconds: Long): String = { + val seconds = milliseconds.toDouble / 1000 + if (seconds < 60) { + return "%.0f s".format(seconds) + } + val minutes = seconds / 60 + if (minutes < 10) { + return "%.1f min".format(minutes) + } else if (minutes < 60) { + return "%.0f min".format(minutes) + } + val hours = minutes / 60 + "%.1f h".format(hours) + } + + /** Generate a verbose human-readable string representing a duration such as "5 second 35 ms" */ + def formatDurationVerbose(ms: Long): String = { + try { + val second = 1000L + val minute = 60 * second + val hour = 60 * minute + val day = 24 * hour + val week = 7 * day + val year = 365 * day + + def toString(num: Long, unit: String): String = { + if (num == 0) { + "" + } else if (num == 1) { + s"$num $unit" + } else { + s"$num ${unit}s" + } + } + + val millisecondsString = if (ms >= second && ms % second == 0) "" else s"${ms % second} ms" + val secondString = toString((ms % minute) / second, "second") + val minuteString = toString((ms % hour) / minute, "minute") + val hourString = toString((ms % day) / hour, "hour") + val dayString = toString((ms % week) / day, "day") + val weekString = toString((ms % year) / week, "week") + val yearString = toString(ms / year, "year") - import Page._ + Seq( + second -> millisecondsString, + minute -> s"$secondString $millisecondsString", + hour -> s"$minuteString $secondString", + day -> s"$hourString $minuteString $secondString", + week -> s"$dayString $hourString $minuteString", + year -> s"$weekString $dayString $hourString" + ).foreach { case (durationLimit, durationString) => + if (ms < durationLimit) { + // if time is less than the limit (upto year) + return durationString + } + } + // if time is more than a year + return s"$yearString $weekString $dayString" + } catch { + case e: Exception => + logError("Error converting time to string", e) + // if there is some error, return blank string + return "" + } + } + + /** Generate a human-readable string representing a number (e.g. 100 K) */ + def formatNumber(records: Double): String = { + val trillion = 1e12 + val billion = 1e9 + val million = 1e6 + val thousand = 1e3 + + val (value, unit) = { + if (records >= 2*trillion) { + (records / trillion, " T") + } else if (records >= 2*billion) { + (records / billion, " B") + } else if (records >= 2*million) { + (records / million, " M") + } else if (records >= 2*thousand) { + (records / thousand, " K") + } else { + (records, "") + } + } + "%.1f%s".formatLocal(Locale.US, value, unit) + } // Yarn has to go through a proxy so the base uri is provided and has to be on all links - private[spark] val uiRoot : String = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")). - getOrElse("") + val uiRoot : String = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")).getOrElse("") def prependBaseUri(basePath: String = "", resource: String = "") = uiRoot + basePath + resource @@ -36,26 +135,14 @@ private[spark] object UIUtils { basePath: String, appName: String, title: String, - page: Page.Value) : Seq[Node] = { - val jobs = page match { - case Stages => -
  • Stages
  • - case _ =>
  • Stages
  • - } - val storage = page match { - case Storage => -
  • Storage
  • - case _ =>
  • Storage
  • - } - val environment = page match { - case Environment => -
  • Environment
  • - case _ =>
  • Environment
  • - } - val executors = page match { - case Executors => -
  • Executors
  • - case _ =>
  • Executors
  • + tabs: Seq[WebUITab], + activeTab: WebUITab, + refreshInterval: Option[Int] = None): Seq[Node] = { + + val header = tabs.map { tab => +
  • + {tab.name} +
  • } @@ -74,16 +161,10 @@ private[spark] object UIUtils { - + -
    @@ -129,21 +210,36 @@ private[spark] object UIUtils { /** Returns an HTML table constructed by generating a row for each object in a sequence. */ def listingTable[T]( headers: Seq[String], - makeRow: T => Seq[Node], - rows: Seq[T], + generateDataRow: T => Seq[Node], + data: Seq[T], fixedWidth: Boolean = false): Seq[Node] = { - val colWidth = 100.toDouble / headers.size - val colWidthAttr = if (fixedWidth) colWidth + "%" else "" var tableClass = "table table-bordered table-striped table-condensed sortable" if (fixedWidth) { tableClass += " table-fixed" } - + val colWidth = 100.toDouble / headers.size + val colWidthAttr = if (fixedWidth) colWidth + "%" else "" + val headerRow: Seq[Node] = { + // if none of the headers have "\n" in them + if (headers.forall(!_.contains("\n"))) { + // represent header as simple text + headers.map(h => {h}) + } else { + // represent header text as list while respecting "\n" + headers.map { case h => + +
      + { h.split("\n").map { case t =>
    • {t}
    • } } +
    + + } + } + } - {headers.map(h => )} + {headerRow} - {rows.map(r => makeRow(r))} + {data.map(r => generateDataRow(r))}
    {h}
    } diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 2cc7582eca8a3..b08f308fda1dd 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -17,53 +17,134 @@ package org.apache.spark.ui -import java.text.SimpleDateFormat -import java.util.Date +import javax.servlet.http.HttpServletRequest -private[spark] abstract class WebUI(name: String) { +import scala.collection.mutable.ArrayBuffer +import scala.xml.Node + +import org.eclipse.jetty.servlet.ServletContextHandler +import org.json4s.JsonAST.{JNothing, JValue} + +import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.Utils + +/** + * The top level component of the UI hierarchy that contains the server. + * + * Each WebUI represents a collection of tabs, each of which in turn represents a collection of + * pages. The use of tabs is optional, however; a WebUI may choose to include pages directly. + */ +private[spark] abstract class WebUI( + securityManager: SecurityManager, + port: Int, + conf: SparkConf, + basePath: String = "") + extends Logging { + + protected val tabs = ArrayBuffer[WebUITab]() + protected val handlers = ArrayBuffer[ServletContextHandler]() protected var serverInfo: Option[ServerInfo] = None + protected val localHostName = Utils.localHostName() + protected val publicHostName = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName) + private val className = Utils.getFormattedClassName(this) + + def getTabs: Seq[WebUITab] = tabs.toSeq + def getHandlers: Seq[ServletContextHandler] = handlers.toSeq + + /** Attach a tab to this UI, along with all of its attached pages. */ + def attachTab(tab: WebUITab) { + tab.pages.foreach(attachPage) + tabs += tab + } + + /** Attach a page to this UI. */ + def attachPage(page: WebUIPage) { + val pagePath = "/" + page.prefix + attachHandler(createServletHandler(pagePath, + (request: HttpServletRequest) => page.render(request), securityManager, basePath)) + attachHandler(createServletHandler(pagePath.stripSuffix("/") + "/json", + (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath)) + } + + /** Attach a handler to this UI. */ + def attachHandler(handler: ServletContextHandler) { + handlers += handler + serverInfo.foreach { info => + info.rootHandler.addHandler(handler) + if (!handler.isStarted) { + handler.start() + } + } + } - /** - * Bind to the HTTP server behind this web interface. - * Overridden implementation should set serverInfo. - */ - def bind() { } + /** Detach a handler from this UI. */ + def detachHandler(handler: ServletContextHandler) { + handlers -= handler + serverInfo.foreach { info => + info.rootHandler.removeHandler(handler) + if (handler.isStarted) { + handler.stop() + } + } + } + + /** Initialize all components of the server. */ + def initialize() + + /** Bind to the HTTP server behind this web interface. */ + def bind() { + assert(!serverInfo.isDefined, "Attempted to bind %s more than once!".format(className)) + try { + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf)) + logInfo("Started %s at http://%s:%d".format(className, publicHostName, boundPort)) + } catch { + case e: Exception => + logError("Failed to bind %s".format(className), e) + System.exit(1) + } + } /** Return the actual port to which this server is bound. Only valid after bind(). */ def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) /** Stop the server behind this web interface. Only valid after bind(). */ def stop() { - assert(serverInfo.isDefined, "Attempted to stop %s before binding to a server!".format(name)) + assert(serverInfo.isDefined, + "Attempted to stop %s before binding to a server!".format(className)) serverInfo.get.server.stop() } } + /** - * Utilities used throughout the web UI. + * A tab that represents a collection of pages. + * The prefix is appended to the parent address to form a full path, and must not contain slashes. */ -private[spark] object WebUI { - // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. - private val dateFormat = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") +private[spark] abstract class WebUITab(parent: WebUI, val prefix: String) { + val pages = ArrayBuffer[WebUIPage]() + val name = prefix.capitalize + + /** Attach a page to this tab. This prepends the page's prefix with the tab's own prefix. */ + def attachPage(page: WebUIPage) { + page.prefix = (prefix + "/" + page.prefix).stripSuffix("/") + pages += page } - def formatDate(date: Date): String = dateFormat.get.format(date) + /** Get a list of header tabs from the parent UI. */ + def headerTabs: Seq[WebUITab] = parent.getTabs +} - def formatDate(timestamp: Long): String = dateFormat.get.format(new Date(timestamp)) - def formatDuration(milliseconds: Long): String = { - val seconds = milliseconds.toDouble / 1000 - if (seconds < 60) { - return "%.0f s".format(seconds) - } - val minutes = seconds / 60 - if (minutes < 10) { - return "%.1f min".format(minutes) - } else if (minutes < 60) { - return "%.0f min".format(minutes) - } - val hours = minutes / 60 - "%.1f h".format(hours) - } +/** + * A page that represents the leaf node in the UI hierarchy. + * + * The direct parent of a WebUIPage is not specified as it can be either a WebUI or a WebUITab. + * If the parent is a WebUI, the prefix is appended to the parent's address to form a full path. + * Else, if the parent is a WebUITab, the prefix is appended to the super prefix of the parent + * to form a relative path. The prefix must not contain slashes. + */ +private[spark] abstract class WebUIPage(var prefix: String) { + def render(request: HttpServletRequest): Seq[Node] + def renderJson(request: HttpServletRequest): JValue = JNothing } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala similarity index 61% rename from core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala rename to core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala index 33df97187ea78..b347eb1b83c1f 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala @@ -21,29 +21,12 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.eclipse.jetty.servlet.ServletContextHandler +import org.apache.spark.ui.{UIUtils, WebUIPage} -import org.apache.spark.scheduler._ -import org.apache.spark.ui._ -import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.Page.Environment - -private[ui] class EnvironmentUI(parent: SparkUI) { +private[ui] class EnvironmentPage(parent: EnvironmentTab) extends WebUIPage("") { + private val appName = parent.appName private val basePath = parent.basePath - private var _listener: Option[EnvironmentListener] = None - - private def appName = parent.appName - - lazy val listener = _listener.get - - def start() { - _listener = Some(new EnvironmentListener) - } - - def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/environment", - (request: HttpServletRequest) => render(request), parent.securityManager, basePath) - ) + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { val runtimeInformationTable = UIUtils.listingTable( @@ -62,7 +45,7 @@ private[ui] class EnvironmentUI(parent: SparkUI) {

    Classpath Entries

    {classpathEntriesTable} - UIUtils.headerSparkPage(content, basePath, appName, "Environment", Environment) + UIUtils.headerSparkPage(content, basePath, appName, "Environment", parent.headerTabs, parent) } private def propertyHeader = Seq("Name", "Value") @@ -71,23 +54,3 @@ private[ui] class EnvironmentUI(parent: SparkUI) { private def propertyRow(kv: (String, String)) = {kv._1}{kv._2} private def classPathRow(data: (String, String)) = {data._1}{data._2} } - -/** - * A SparkListener that prepares information to be displayed on the EnvironmentUI - */ -private[ui] class EnvironmentListener extends SparkListener { - var jvmInformation = Seq[(String, String)]() - var sparkProperties = Seq[(String, String)]() - var systemProperties = Seq[(String, String)]() - var classpathEntries = Seq[(String, String)]() - - override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { - synchronized { - val environmentDetails = environmentUpdate.environmentDetails - jvmInformation = environmentDetails("JVM Information") - sparkProperties = environmentDetails("Spark Properties") - systemProperties = environmentDetails("System Properties") - classpathEntries = environmentDetails("Classpath Entries") - } - } -} diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala new file mode 100644 index 0000000000000..03b46e1bd59af --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala @@ -0,0 +1,50 @@ +/* + * 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.ui.env + +import org.apache.spark.scheduler._ +import org.apache.spark.ui._ + +private[ui] class EnvironmentTab(parent: SparkUI) extends WebUITab(parent, "environment") { + val appName = parent.appName + val basePath = parent.basePath + val listener = new EnvironmentListener + + attachPage(new EnvironmentPage(this)) + parent.registerListener(listener) +} + +/** + * A SparkListener that prepares information to be displayed on the EnvironmentTab + */ +private[ui] class EnvironmentListener extends SparkListener { + var jvmInformation = Seq[(String, String)]() + var sparkProperties = Seq[(String, String)]() + var systemProperties = Seq[(String, String)]() + var classpathEntries = Seq[(String, String)]() + + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { + synchronized { + val environmentDetails = environmentUpdate.environmentDetails + jvmInformation = environmentDetails("JVM Information") + sparkProperties = environmentDetails("Spark Properties") + systemProperties = environmentDetails("System Properties") + classpathEntries = environmentDetails("Classpath Entries") + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala similarity index 61% rename from core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala rename to core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 77a38a1d3aa7c..c1e69f6cdaffb 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -19,35 +19,15 @@ package org.apache.spark.ui.exec import javax.servlet.http.HttpServletRequest -import scala.collection.mutable.HashMap import scala.xml.Node -import org.eclipse.jetty.servlet.ServletContextHandler - -import org.apache.spark.ExceptionFailure -import org.apache.spark.scheduler._ -import org.apache.spark.storage.StorageStatusListener -import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.Page.Executors -import org.apache.spark.ui.{SparkUI, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[ui] class ExecutorsUI(parent: SparkUI) { +private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { + private val appName = parent.appName private val basePath = parent.basePath - private var _listener: Option[ExecutorsListener] = None - - private def appName = parent.appName - - lazy val listener = _listener.get - - def start() { - _listener = Some(new ExecutorsListener(parent.storageStatusListener)) - } - - def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/executors", - (request: HttpServletRequest) => render(request), parent.securityManager, basePath) - ) + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = listener.storageStatusList @@ -75,8 +55,8 @@ private[ui] class ExecutorsUI(parent: SparkUI) {
    ; - UIUtils.headerSparkPage( - content, basePath, appName, "Executors (" + execInfo.size + ")", Executors) + UIUtils.headerSparkPage(content, basePath, appName, "Executors (" + execInfo.size + ")", + parent.headerTabs, parent) } /** Header fields for the executors table */ @@ -159,55 +139,3 @@ private[ui] class ExecutorsUI(parent: SparkUI) { execFields.zip(execValues).toMap } } - -/** - * A SparkListener that prepares information to be displayed on the ExecutorsUI - */ -private[ui] class ExecutorsListener(storageStatusListener: StorageStatusListener) - extends SparkListener { - - val executorToTasksActive = HashMap[String, Int]() - val executorToTasksComplete = HashMap[String, Int]() - val executorToTasksFailed = HashMap[String, Int]() - val executorToDuration = HashMap[String, Long]() - val executorToShuffleRead = HashMap[String, Long]() - val executorToShuffleWrite = HashMap[String, Long]() - - def storageStatusList = storageStatusListener.storageStatusList - - override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { - val eid = formatExecutorId(taskStart.taskInfo.executorId) - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 - } - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { - val info = taskEnd.taskInfo - if (info != null) { - val eid = formatExecutorId(info.executorId) - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 - executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + info.duration - taskEnd.reason match { - case e: ExceptionFailure => - executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 - case _ => - executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 - } - - // Update shuffle read/write - val metrics = taskEnd.taskMetrics - if (metrics != null) { - metrics.shuffleReadMetrics.foreach { shuffleRead => - executorToShuffleRead(eid) = - executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead - } - metrics.shuffleWriteMetrics.foreach { shuffleWrite => - executorToShuffleWrite(eid) = - executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten - } - } - } - } - - // This addresses executor ID inconsistencies in the local mode - private def formatExecutorId(execId: String) = storageStatusListener.formatExecutorId(execId) -} diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala new file mode 100644 index 0000000000000..5678bf34ac730 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -0,0 +1,86 @@ +/* + * 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.ui.exec + +import scala.collection.mutable.HashMap + +import org.apache.spark.ExceptionFailure +import org.apache.spark.scheduler._ +import org.apache.spark.storage.StorageStatusListener +import org.apache.spark.ui.{SparkUI, WebUITab} + +private[ui] class ExecutorsTab(parent: SparkUI) extends WebUITab(parent, "executors") { + val appName = parent.appName + val basePath = parent.basePath + val listener = new ExecutorsListener(parent.storageStatusListener) + + attachPage(new ExecutorsPage(this)) + parent.registerListener(listener) +} + +/** + * A SparkListener that prepares information to be displayed on the ExecutorsTab + */ +private[ui] class ExecutorsListener(storageStatusListener: StorageStatusListener) + extends SparkListener { + + val executorToTasksActive = HashMap[String, Int]() + val executorToTasksComplete = HashMap[String, Int]() + val executorToTasksFailed = HashMap[String, Int]() + val executorToDuration = HashMap[String, Long]() + val executorToShuffleRead = HashMap[String, Long]() + val executorToShuffleWrite = HashMap[String, Long]() + + def storageStatusList = storageStatusListener.storageStatusList + + override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { + val eid = formatExecutorId(taskStart.taskInfo.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + val info = taskEnd.taskInfo + if (info != null) { + val eid = formatExecutorId(info.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 + executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + info.duration + taskEnd.reason match { + case e: ExceptionFailure => + executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 + case _ => + executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 + } + + // Update shuffle read/write + val metrics = taskEnd.taskMetrics + if (metrics != null) { + metrics.shuffleReadMetrics.foreach { shuffleRead => + executorToShuffleRead(eid) = + executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead + } + metrics.shuffleWriteMetrics.foreach { shuffleWrite => + executorToShuffleWrite(eid) = + executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten + } + } + } + } + + // This addresses executor ID inconsistencies in the local mode + private def formatExecutorId(execId: String) = storageStatusListener.formatExecutorId(execId) +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 73861ae6746da..c83e196c9c156 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -20,11 +20,12 @@ package org.apache.spark.ui.jobs import scala.collection.mutable import scala.xml.Node +import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing executor summary */ -private[ui] class ExecutorTable(stageId: Int, parent: JobProgressUI) { - private lazy val listener = parent.listener +private[ui] class ExecutorTable(stageId: Int, parent: JobProgressTab) { + private val listener = parent.listener def toNodeSeq: Seq[Node] = { listener.synchronized { @@ -69,7 +70,7 @@ private[ui] class ExecutorTable(stageId: Int, parent: JobProgressUI) { {k} {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")} - {parent.formatDuration(v.taskTime)} + {UIUtils.formatDuration(v.taskTime)} {v.failedTasks + v.succeededTasks} {v.failedTasks} {v.succeededTasks} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 5167e20ea3d7d..0db4afa701b41 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -222,12 +222,10 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { synchronized { - val schedulingModeName = - environmentUpdate.environmentDetails("Spark Properties").toMap.get("spark.scheduler.mode") - schedulingMode = schedulingModeName match { - case Some(name) => Some(SchedulingMode.withName(name)) - case None => None - } + schedulingMode = environmentUpdate + .environmentDetails("Spark Properties").toMap + .get("spark.scheduler.mode") + .map(SchedulingMode.withName) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala similarity index 90% rename from core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala index 8619a31380f1e..34ff2ac34a7ca 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala @@ -22,25 +22,23 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, NodeSeq} import org.apache.spark.scheduler.Schedulable -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing list of all ongoing and recently finished stages and pools */ -private[ui] class IndexPage(parent: JobProgressUI) { +private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") { + private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc - private lazy val listener = parent.listener + private val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler - private def appName = parent.appName - def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val activeStages = listener.activeStages.values.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq - val now = System.currentTimeMillis() + val now = System.currentTimeMillis val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent, parent.killEnabled) @@ -59,7 +57,7 @@ private[ui] class IndexPage(parent: JobProgressUI) { // Total duration is not meaningful unless the UI is live
  • Total Duration: - {parent.formatDuration(now - sc.startTime)} + {UIUtils.formatDuration(now - sc.startTime)}
  • }}
  • @@ -94,7 +92,7 @@ private[ui] class IndexPage(parent: JobProgressUI) {

    Failed Stages ({failedStages.size})

    ++ failedStagesTable.toNodeSeq - UIUtils.headerSparkPage(content, basePath, appName, "Spark Stages", Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Spark Stages", parent.headerTabs, parent) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala similarity index 53% rename from core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala index 30e3f35f2182b..3308c8c8a3d37 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala @@ -19,39 +19,28 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.servlet.ServletContextHandler - import org.apache.spark.SparkConf import org.apache.spark.scheduler.SchedulingMode -import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.SparkUI -import org.apache.spark.util.Utils +import org.apache.spark.ui.{SparkUI, WebUITab} /** Web UI showing progress status of all jobs in the given SparkContext. */ -private[ui] class JobProgressUI(parent: SparkUI) { +private[ui] class JobProgressTab(parent: SparkUI) extends WebUITab(parent, "stages") { + val appName = parent.appName val basePath = parent.basePath val live = parent.live val sc = parent.sc - val killEnabled = parent.conf.getBoolean("spark.ui.killEnabled", true) - - lazy val listener = _listener.get - lazy val isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) - - private val indexPage = new IndexPage(this) - private val stagePage = new StagePage(this) - private val poolPage = new PoolPage(this) - private var _listener: Option[JobProgressListener] = None + val conf = if (live) sc.conf else new SparkConf + val killEnabled = conf.getBoolean("spark.ui.killEnabled", true) + val listener = new JobProgressListener(conf) - def appName = parent.appName + attachPage(new JobProgressPage(this)) + attachPage(new StagePage(this)) + attachPage(new PoolPage(this)) + parent.registerListener(listener) - def start() { - val conf = if (live) sc.conf else new SparkConf - _listener = Some(new JobProgressListener(conf)) - } - - def formatDuration(ms: Long) = Utils.msDurationToString(ms) + def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) - private def handleKillRequest(request: HttpServletRequest) = { + def handleKillRequest(request: HttpServletRequest) = { if (killEnabled) { val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt @@ -64,14 +53,4 @@ private[ui] class JobProgressUI(parent: SparkUI) { Thread.sleep(100) } } - - def getHandlers = Seq[ServletContextHandler]( - createRedirectHandler("/stages/stage/kill", "/stages", handleKillRequest), - createServletHandler("/stages/stage", - (request: HttpServletRequest) => stagePage.render(request), parent.securityManager, basePath), - createServletHandler("/stages/pool", - (request: HttpServletRequest) => poolPage.render(request), parent.securityManager, basePath), - createServletHandler("/stages", - (request: HttpServletRequest) => indexPage.render(request), parent.securityManager, basePath) - ) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 3638e6035ba81..fd83d37583967 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -22,17 +22,15 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.scheduler.{Schedulable, StageInfo} -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing specific pool details */ -private[ui] class PoolPage(parent: JobProgressUI) { +private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { + private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc - private lazy val listener = parent.listener - - private def appName = parent.appName + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { @@ -52,8 +50,8 @@ private[ui] class PoolPage(parent: JobProgressUI) {

    Summary

    ++ poolTable.toNodeSeq ++

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq - UIUtils.headerSparkPage( - content, basePath, appName, "Fair Scheduler Pool: " + poolName, Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Fair Scheduler Pool: " + poolName, + parent.headerTabs, parent) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index c5c8d8668740b..f4b68f241966d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -24,10 +24,9 @@ import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.UIUtils /** Table showing list of pools */ -private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { +private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressTab) { private val basePath = parent.basePath - private val poolToActiveStages = listener.poolToActiveStages - private lazy val listener = parent.listener + private val listener = parent.listener def toNodeSeq: Seq[Node] = { listener.synchronized { @@ -48,7 +47,7 @@ private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { SchedulingMode - {rows.map(r => makeRow(r, poolToActiveStages))} + {rows.map(r => makeRow(r, listener.poolToActiveStages))} } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index b6c3e3cf45163..4bce472036f7d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -22,17 +22,14 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.{WebUI, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ -private[ui] class StagePage(parent: JobProgressUI) { +private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { + private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.listener - private lazy val sc = parent.sc - - private def appName = parent.appName + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { @@ -44,8 +41,8 @@ private[ui] class StagePage(parent: JobProgressUI) {

    Summary Metrics

    No tasks have started yet

    Tasks

    No tasks have started yet
  • - return UIUtils.headerSparkPage( - content, basePath, appName, "Details for Stage %s".format(stageId), Stages) + return UIUtils.headerSparkPage(content, basePath, appName, + "Details for Stage %s".format(stageId), parent.headerTabs, parent) } val tasks = listener.stageIdToTaskData(stageId).values.toSeq.sortBy(_.taskInfo.launchTime) @@ -60,7 +57,7 @@ private[ui] class StagePage(parent: JobProgressUI) { val hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0 var activeTime = 0L - val now = System.currentTimeMillis() + val now = System.currentTimeMillis val tasksActive = listener.stageIdToTasksActive(stageId).values tasksActive.foreach(activeTime += _.timeRunning(now)) @@ -70,7 +67,7 @@ private[ui] class StagePage(parent: JobProgressUI) {
    • Total task time across all tasks: - {parent.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)} + {UIUtils.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)}
    • {if (hasShuffleRead)
    • @@ -121,13 +118,13 @@ private[ui] class StagePage(parent: JobProgressUI) { } val serializationQuantiles = "Result serialization time" +: Distribution(serializationTimes). - get.getQuantiles().map(ms => parent.formatDuration(ms.toLong)) + get.getQuantiles().map(ms => UIUtils.formatDuration(ms.toLong)) val serviceTimes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.executorRunTime.toDouble } val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles() - .map(ms => parent.formatDuration(ms.toLong)) + .map(ms => UIUtils.formatDuration(ms.toLong)) val gettingResultTimes = validTasks.map { case TaskUIData(info, _, _) => if (info.gettingResultTime > 0) { @@ -138,7 +135,7 @@ private[ui] class StagePage(parent: JobProgressUI) { } val gettingResultQuantiles = "Time spent fetching task results" +: Distribution(gettingResultTimes).get.getQuantiles().map { millis => - parent.formatDuration(millis.toLong) + UIUtils.formatDuration(millis.toLong) } // The scheduler delay includes the network delay to send the task to the worker // machine and to send back the result (but not the time to fetch the task result, @@ -155,7 +152,7 @@ private[ui] class StagePage(parent: JobProgressUI) { } val schedulerDelayQuantiles = "Scheduler delay" +: Distribution(schedulerDelays).get.getQuantiles().map { millis => - parent.formatDuration(millis.toLong) + UIUtils.formatDuration(millis.toLong) } def getQuantileCols(data: Seq[Double]) = @@ -206,8 +203,8 @@ private[ui] class StagePage(parent: JobProgressUI) {

      Aggregated Metrics by Executor

      ++ executorTable.toNodeSeq ++

      Tasks

      ++ taskTable - UIUtils.headerSparkPage( - content, basePath, appName, "Details for Stage %d".format(stageId), Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Details for Stage %d".format(stageId), + parent.headerTabs, parent) } } @@ -219,8 +216,8 @@ private[ui] class StagePage(parent: JobProgressUI) { taskData match { case TaskUIData(info, metrics, exception) => val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) else metrics.map(_.executorRunTime).getOrElse(1L) - val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) - else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") + val formatDuration = if (info.status == "RUNNING") UIUtils.formatDuration(duration) + else metrics.map(m => UIUtils.formatDuration(m.executorRunTime)).getOrElse("") val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) @@ -235,8 +232,8 @@ private[ui] class StagePage(parent: JobProgressUI) { val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleWriteTime) val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") - val writeTimeReadable = maybeWriteTime.map( t => t / (1000 * 1000)).map { ms => - if (ms == 0) "" else parent.formatDuration(ms) + val writeTimeReadable = maybeWriteTime.map(t => t / (1000 * 1000)).map { ms => + if (ms == 0) "" else UIUtils.formatDuration(ms) }.getOrElse("") val maybeMemoryBytesSpilled = metrics.map(_.memoryBytesSpilled) @@ -254,15 +251,15 @@ private[ui] class StagePage(parent: JobProgressUI) { {info.status} {info.taskLocality} {info.host} - {WebUI.formatDate(new Date(info.launchTime))} + {UIUtils.formatDate(new Date(info.launchTime))} {formatDuration} - {if (gcTime > 0) parent.formatDuration(gcTime) else ""} + {if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""} - {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""} + {if (serializationTime > 0) UIUtils.formatDuration(serializationTime) else ""} {if (shuffleRead) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index e419fae5a6589..8c5b1f55fd2dc 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -23,17 +23,17 @@ import scala.collection.mutable.HashMap import scala.xml.Node import org.apache.spark.scheduler.{StageInfo, TaskInfo} -import org.apache.spark.ui.{WebUI, UIUtils} +import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ private[ui] class StageTable( - stages: Seq[StageInfo], - parent: JobProgressUI, - killEnabled: Boolean = false) { + stages: Seq[StageInfo], + parent: JobProgressTab, + killEnabled: Boolean = false) { private val basePath = parent.basePath - private lazy val listener = parent.listener + private val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler def toNodeSeq: Seq[Node] = { @@ -89,25 +89,23 @@ private[ui] class StageTable( {s.name} - val description = listener.stageIdToDescription.get(s.stageId) + listener.stageIdToDescription.get(s.stageId) .map(d =>
      {d}
      {nameLink} {killLink}
      ) .getOrElse(
      {killLink}{nameLink}
      ) - - return description } /** Render an HTML row that represents a stage */ private def stageRow(s: StageInfo): Seq[Node] = { val poolName = listener.stageIdToPool.get(s.stageId) val submissionTime = s.submissionTime match { - case Some(t) => WebUI.formatDate(new Date(t)) + case Some(t) => UIUtils.formatDate(new Date(t)) case None => "Unknown" } val finishTime = s.completionTime.getOrElse(System.currentTimeMillis) val duration = s.submissionTime.map { t => if (finishTime > t) finishTime - t else System.currentTimeMillis - t } - val formattedDuration = duration.map(d => parent.formatDuration(d)).getOrElse("Unknown") + val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") val startedTasks = listener.stageIdToTasksActive.getOrElse(s.stageId, HashMap[Long, TaskInfo]()).size val completedTasks = listener.stageIdToTasksComplete.getOrElse(s.stageId, 0) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 75ee9976d7b5f..d07f1c9b20fcf 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -22,23 +22,22 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.{BlockId, BlockStatus, StorageStatus, StorageUtils} -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ -private[ui] class RDDPage(parent: BlockManagerUI) { +private[ui] class RddPage(parent: StorageTab) extends WebUIPage("rdd") { + private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.listener - - private def appName = parent.appName + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { val rddId = request.getParameter("id").toInt val storageStatusList = listener.storageStatusList val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse { // Rather than crashing, render an "RDD Not Found" page - return UIUtils.headerSparkPage(Seq[Node](), basePath, appName, "RDD Not Found", Storage) + return UIUtils.headerSparkPage(Seq[Node](), basePath, appName, "RDD Not Found", + parent.headerTabs, parent) } // Worker table @@ -96,8 +95,8 @@ private[ui] class RDDPage(parent: BlockManagerUI) { ; - UIUtils.headerSparkPage( - content, basePath, appName, "RDD Storage Info for " + rddInfo.name, Storage) + UIUtils.headerSparkPage(content, basePath, appName, "RDD Storage Info for " + rddInfo.name, + parent.headerTabs, parent) } /** Header fields for the worker table */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala similarity index 90% rename from core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala rename to core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 4f6acc30a88c4..b66edd91f56c0 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -22,22 +22,19 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.RDDInfo -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ -private[ui] class IndexPage(parent: BlockManagerUI) { +private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { + private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.listener - - private def appName = parent.appName + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - val rdds = listener.rddInfoList val content = UIUtils.listingTable(rddHeader, rddRow, rdds) - UIUtils.headerSparkPage(content, basePath, appName, "Storage ", Storage) + UIUtils.headerSparkPage(content, basePath, appName, "Storage ", parent.headerTabs, parent) } /** Header fields for the RDD table */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala similarity index 75% rename from core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala rename to core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 16996a2da1e72..56429f6c07fcd 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -17,45 +17,27 @@ package org.apache.spark.ui.storage -import javax.servlet.http.HttpServletRequest - import scala.collection.mutable -import org.eclipse.jetty.servlet.ServletContextHandler - import org.apache.spark.ui._ -import org.apache.spark.ui.JettyUtils._ import org.apache.spark.scheduler._ import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[ui] class BlockManagerUI(parent: SparkUI) { +private[ui] class StorageTab(parent: SparkUI) extends WebUITab(parent, "storage") { + val appName = parent.appName val basePath = parent.basePath + val listener = new StorageListener(parent.storageStatusListener) - private val indexPage = new IndexPage(this) - private val rddPage = new RDDPage(this) - private var _listener: Option[BlockManagerListener] = None - - lazy val listener = _listener.get - - def appName = parent.appName - - def start() { - _listener = Some(new BlockManagerListener(parent.storageStatusListener)) - } - - def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/storage/rdd", - (request: HttpServletRequest) => rddPage.render(request), parent.securityManager, basePath), - createServletHandler("/storage", - (request: HttpServletRequest) => indexPage.render(request), parent.securityManager, basePath) - ) + attachPage(new StoragePage(this)) + attachPage(new RddPage(this)) + parent.registerListener(listener) } /** * A SparkListener that prepares information to be displayed on the BlockManagerUI */ -private[ui] class BlockManagerListener(storageStatusListener: StorageStatusListener) +private[ui] class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener { private val _rddInfoMap = mutable.Map[Int, RDDInfo]() diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index f2396f7c80a35..465835ea7fe29 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -88,30 +88,27 @@ private[spark] object JsonProtocol { def taskStartToJson(taskStart: SparkListenerTaskStart): JValue = { val taskInfo = taskStart.taskInfo - val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing ("Event" -> Utils.getFormattedClassName(taskStart)) ~ ("Stage ID" -> taskStart.stageId) ~ - ("Task Info" -> taskInfoJson) + ("Task Info" -> taskInfoToJson(taskInfo)) } def taskGettingResultToJson(taskGettingResult: SparkListenerTaskGettingResult): JValue = { val taskInfo = taskGettingResult.taskInfo - val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing ("Event" -> Utils.getFormattedClassName(taskGettingResult)) ~ - ("Task Info" -> taskInfoJson) + ("Task Info" -> taskInfoToJson(taskInfo)) } def taskEndToJson(taskEnd: SparkListenerTaskEnd): JValue = { val taskEndReason = taskEndReasonToJson(taskEnd.reason) val taskInfo = taskEnd.taskInfo - val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing val taskMetrics = taskEnd.taskMetrics val taskMetricsJson = if (taskMetrics != null) taskMetricsToJson(taskMetrics) else JNothing ("Event" -> Utils.getFormattedClassName(taskEnd)) ~ ("Stage ID" -> taskEnd.stageId) ~ ("Task Type" -> taskEnd.taskType) ~ ("Task End Reason" -> taskEndReason) ~ - ("Task Info" -> taskInfoJson) ~ + ("Task Info" -> taskInfoToJson(taskInfo)) ~ ("Task Metrics" -> taskMetricsJson) } @@ -505,6 +502,9 @@ private[spark] object JsonProtocol { } def taskMetricsFromJson(json: JValue): TaskMetrics = { + if (json == JNothing) { + return TaskMetrics.empty + } val metrics = new TaskMetrics metrics.hostname = (json \ "Host Name").extract[String] metrics.executorDeserializeTime = (json \ "Executor Deserialize Time").extract[Long] diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 2f9739f940dc6..b85c483ca2a08 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -18,16 +18,81 @@ package org.apache.spark.ui import java.net.ServerSocket +import javax.servlet.http.HttpServletRequest +import scala.io.Source import scala.util.{Failure, Success, Try} import org.eclipse.jetty.server.Server import org.eclipse.jetty.servlet.ServletContextHandler import org.scalatest.FunSuite +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ -import org.apache.spark.SparkConf +import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.LocalSparkContext._ +import scala.xml.Node class UISuite extends FunSuite { + + test("basic ui visibility") { + withSpark(new SparkContext("local", "test")) { sc => + // test if the ui is visible, and all the expected tabs are visible + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(sc.ui.appUIAddress).mkString + assert(!html.contains("random data that should not be present")) + assert(html.toLowerCase.contains("stages")) + assert(html.toLowerCase.contains("storage")) + assert(html.toLowerCase.contains("environment")) + assert(html.toLowerCase.contains("executors")) + } + } + } + + test("visibility at localhost:4040") { + withSpark(new SparkContext("local", "test")) { sc => + // test if visible from http://localhost:4040 + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL("http://localhost:4040").mkString + assert(html.toLowerCase.contains("stages")) + } + } + } + + test("attaching a new tab") { + withSpark(new SparkContext("local", "test")) { sc => + val sparkUI = sc.ui + + val newTab = new WebUITab(sparkUI, "foo") { + attachPage(new WebUIPage("") { + def render(request: HttpServletRequest): Seq[Node] = { + "html magic" + } + }) + } + sparkUI.attachTab(newTab) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(sc.ui.appUIAddress).mkString + assert(!html.contains("random data that should not be present")) + + // check whether new page exists + assert(html.toLowerCase.contains("foo")) + + // check whether other pages still exist + assert(html.toLowerCase.contains("stages")) + assert(html.toLowerCase.contains("storage")) + assert(html.toLowerCase.contains("environment")) + assert(html.toLowerCase.contains("executors")) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(sc.ui.appUIAddress.stripSuffix("/") + "/foo").mkString + // check whether new page exists + assert(html.contains("magic")) + } + } + } + test("jetty port increases under contention") { val startPort = 4040 val server = new Server(startPort) @@ -60,4 +125,18 @@ class UISuite extends FunSuite { case Failure(e) => } } + + test("verify appUIAddress contains the scheme") { + withSpark(new SparkContext("local", "test")) { sc => + val uiAddress = sc.ui.appUIAddress + assert(uiAddress.equals("http://" + sc.ui.appUIHostPort)) + } + } + + test("verify appUIAddress contains the port") { + withSpark(new SparkContext("local", "test")) { sc => + val splitUIAddress = sc.ui.appUIAddress.split(':') + assert(splitUIAddress(2).toInt == sc.ui.boundPort) + } + } } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index f75297a02dc8b..16470bb7bf60d 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -523,8 +523,8 @@ class JsonProtocolSuite extends FunSuite { 700,"Fetch Wait Time":900,"Remote Bytes Read":1000},"Shuffle Write Metrics": {"Shuffle Bytes Written":1200,"Shuffle Write Time":1500},"Updated Blocks": [{"Block ID":{"Type":"RDDBlockId","RDD ID":0,"Split Index":0},"Status": - {"Storage Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":false, - "Replication":2},"Memory Size":0,"Disk Size":0,"Tachyon Size":0}}]}} + {"Storage Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false, + "Deserialized":false,"Replication":2},"Memory Size":0,"Disk Size":0,"Tachyon Size":0}}]}} """ private val jobStartJsonString = diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 5ea4817bfde18..9cb31d70444ff 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -60,6 +60,7 @@ object MimaBuild { Seq( excludePackage("org.apache.spark.api.java"), excludePackage("org.apache.spark.streaming.api.java"), + excludePackage("org.apache.spark.streaming.scheduler"), excludePackage("org.apache.spark.mllib") ) ++ excludeSparkClass("rdd.ClassTags") ++ @@ -70,7 +71,12 @@ object MimaBuild { excludeSparkClass("mllib.regression.LassoWithSGD") ++ excludeSparkClass("mllib.regression.LinearRegressionWithSGD") ++ excludeSparkClass("streaming.dstream.NetworkReceiver") ++ - excludeSparkClass("streaming.dstream.NetworkReceiver#NetworkReceiverActor") + excludeSparkClass("streaming.dstream.NetworkReceiver#NetworkReceiverActor") ++ + excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator") ++ + excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator#Block") ++ + excludeSparkClass("streaming.dstream.ReportError") ++ + excludeSparkClass("streaming.dstream.ReportBlock") ++ + excludeSparkClass("streaming.dstream.DStream") case _ => Seq() } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index a4e236c65ff86..ff5d0aaa3d0bd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -17,29 +17,28 @@ package org.apache.spark.streaming -import scala.collection.mutable.Queue -import scala.collection.Map -import scala.reflect.ClassTag - import java.io.InputStream import java.util.concurrent.atomic.AtomicInteger -import akka.actor.Props -import akka.actor.SupervisorStrategy -import org.apache.hadoop.io.LongWritable -import org.apache.hadoop.io.Text +import scala.collection.Map +import scala.collection.mutable.Queue +import scala.reflect.ClassTag + +import akka.actor.{Props, SupervisorStrategy} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat -import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.MetadataCleaner import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.receivers._ import org.apache.spark.streaming.scheduler._ -import org.apache.hadoop.conf.Configuration +import org.apache.spark.streaming.ui.StreamingTab +import org.apache.spark.util.MetadataCleaner /** * Main entry point for Spark Streaming functionality. It provides methods used to create @@ -158,6 +157,8 @@ class StreamingContext private[streaming] ( private[streaming] val waiter = new ContextWaiter + private[streaming] val uiTab = new StreamingTab(this) + /** Enumeration to identify current state of the StreamingContext */ private[streaming] object StreamingContextState extends Enumeration { type CheckpointState = Value diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index d043200f71a0b..a7e5215437e54 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -353,15 +353,6 @@ abstract class DStream[T: ClassTag] ( dependencies.foreach(_.clearMetadata(time)) } - /* Adds metadata to the Stream while it is running. - * This method should be overwritten by sublcasses of InputDStream. - */ - private[streaming] def addMetadata(metadata: Any) { - if (metadata != null) { - logInfo("Dropping Metadata: " + metadata.toString) - } - } - /** * Refresh the list of checkpointed RDDs that will be saved along with checkpoint of * this stream. This is an internal method that should not be called directly. This is diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index d19a635fe8eca..5a249706b4d2f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -17,24 +17,23 @@ package org.apache.spark.streaming.dstream -import java.util.concurrent.{TimeUnit, ArrayBlockingQueue} import java.nio.ByteBuffer +import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.Await -import scala.concurrent.duration._ import scala.reflect.ClassTag -import akka.actor.{Props, Actor} +import akka.actor.{Actor, Props} import akka.pattern.ask -import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} -import org.apache.spark.streaming._ import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.rdd.{RDD, BlockRDD} +import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.{BlockId, StorageLevel, StreamBlockId} -import org.apache.spark.streaming.scheduler.{DeregisterReceiver, AddBlocks, RegisterReceiver} -import org.apache.spark.util.AkkaUtils +import org.apache.spark.streaming._ +import org.apache.spark.streaming.scheduler.{AddBlock, DeregisterReceiver, ReceivedBlockInfo, RegisterReceiver} +import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} +import org.apache.spark.util.{AkkaUtils, Utils} /** * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] @@ -49,8 +48,10 @@ import org.apache.spark.util.AkkaUtils abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingContext) extends InputDStream[T](ssc_) { - // This is an unique identifier that is used to match the network receiver with the - // corresponding network input stream. + /** Keeps all received blocks information */ + private lazy val receivedBlockInfo = new HashMap[Time, Array[ReceivedBlockInfo]] + + /** This is an unique identifier for the network input stream. */ val id = ssc.getNewNetworkStreamId() /** @@ -65,25 +66,44 @@ abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingConte def stop() {} + /** Ask NetworkInputTracker for received data blocks and generates RDDs with them. */ override def compute(validTime: Time): Option[RDD[T]] = { // If this is called for any time before the start time of the context, // then this returns an empty RDD. This may happen when recovering from a // master failure if (validTime >= graph.startTime) { - val blockIds = ssc.scheduler.networkInputTracker.getBlocks(id, validTime) + val blockInfo = ssc.scheduler.networkInputTracker.getReceivedBlockInfo(id) + receivedBlockInfo(validTime) = blockInfo + val blockIds = blockInfo.map(_.blockId.asInstanceOf[BlockId]) Some(new BlockRDD[T](ssc.sc, blockIds)) } else { Some(new BlockRDD[T](ssc.sc, Array[BlockId]())) } } + + /** Get information on received blocks. */ + private[streaming] def getReceivedBlockInfo(time: Time) = { + receivedBlockInfo(time) + } + + /** + * Clear metadata that are older than `rememberDuration` of this DStream. + * This is an internal method that should not be called directly. This + * implementation overrides the default implementation to clear received + * block information. + */ + private[streaming] override def clearMetadata(time: Time) { + super.clearMetadata(time) + val oldReceivedBlocks = receivedBlockInfo.filter(_._1 <= (time - rememberDuration)) + receivedBlockInfo --= oldReceivedBlocks.keys + logDebug("Cleared " + oldReceivedBlocks.size + " RDDs that were older than " + + (time - rememberDuration) + ": " + oldReceivedBlocks.keys.mkString(", ")) + } } private[streaming] sealed trait NetworkReceiverMessage -private[streaming] case class StopReceiver() extends NetworkReceiverMessage -private[streaming] case class ReportBlock(blockId: BlockId, metadata: Any) - extends NetworkReceiverMessage -private[streaming] case class ReportError(msg: String) extends NetworkReceiverMessage +private[streaming] case class StopReceiver(msg: String) extends NetworkReceiverMessage /** * Abstract class of a receiver that can be run on worker nodes to receive external data. See @@ -177,6 +197,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging case (e, i) => "Exception " + i + ": " + e.getMessage + "\n" + e.getStackTraceString }.mkString("\n") } + logInfo("Deregistering receiver " + streamId) val future = trackerActor.ask(DeregisterReceiver(streamId, message))(askTimeout) Await.result(future, askTimeout) @@ -209,18 +230,28 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging /** * Push a block (as an ArrayBuffer filled with data) into the block manager. */ - def pushBlock(blockId: BlockId, arrayBuffer: ArrayBuffer[T], metadata: Any, level: StorageLevel) { + def pushBlock( + blockId: StreamBlockId, + arrayBuffer: ArrayBuffer[T], + metadata: Any, + level: StorageLevel + ) { env.blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], level) - trackerActor ! AddBlocks(streamId, Array(blockId), metadata) + trackerActor ! AddBlock(ReceivedBlockInfo(streamId, blockId, arrayBuffer.size, metadata)) logDebug("Pushed block " + blockId) } /** * Push a block (as bytes) into the block manager. */ - def pushBlock(blockId: BlockId, bytes: ByteBuffer, metadata: Any, level: StorageLevel) { + def pushBlock( + blockId: StreamBlockId, + bytes: ByteBuffer, + metadata: Any, + level: StorageLevel + ) { env.blockManager.putBytes(blockId, bytes, level) - trackerActor ! AddBlocks(streamId, Array(blockId), metadata) + trackerActor ! AddBlock(ReceivedBlockInfo(streamId, blockId, -1, metadata)) } /** Set the ID of the DStream that this receiver is associated with */ @@ -232,9 +263,11 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging private class NetworkReceiverActor extends Actor { override def preStart() { - logInfo("Registered receiver " + streamId) - val future = trackerActor.ask(RegisterReceiver(streamId, self))(askTimeout) + val msg = RegisterReceiver( + streamId, NetworkReceiver.this.getClass.getSimpleName, Utils.localHostName(), self) + val future = trackerActor.ask(msg)(askTimeout) Await.result(future, askTimeout) + logInfo("Registered receiver " + streamId) } override def receive() = { @@ -253,7 +286,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging class BlockGenerator(storageLevel: StorageLevel) extends Serializable with Logging { - case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null) + case class Block(id: StreamBlockId, buffer: ArrayBuffer[T], metadata: Any = null) val clock = new SystemClock() val blockInterval = env.conf.getLong("spark.streaming.blockInterval", 200) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala index 7f3cd2f8eb1fd..9c69a2a4e21f5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala @@ -29,6 +29,7 @@ import org.apache.spark.streaming.Time */ case class BatchInfo( batchTime: Time, + receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]], submissionTime: Long, processingStartTime: Option[Long], processingEndTime: Option[Long] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 92d885c4bc5a5..e564eccba2df5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -201,7 +201,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { logInfo("Batches to reschedule (" + timesToReschedule.size + " batches): " + timesToReschedule.mkString(", ")) timesToReschedule.foreach(time => - jobScheduler.runJobs(time, graph.generateJobs(time)) + jobScheduler.submitJobSet(JobSet(time, graph.generateJobs(time))) ) // Restart the timer @@ -214,7 +214,12 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { SparkEnv.set(ssc.env) Try(graph.generateJobs(time)) match { case Success(jobs) => - jobScheduler.runJobs(time, jobs) + val receivedBlockInfo = graph.getNetworkInputStreams.map { stream => + val streamId = stream.id + val receivedBlockInfo = stream.getReceivedBlockInfo(time) + (streamId, receivedBlockInfo) + }.toMap + jobScheduler.submitJobSet(JobSet(time, jobs, receivedBlockInfo)) case Failure(e) => jobScheduler.reportError("Error generating jobs for time " + time, e) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 04e0a6a283cfb..d9ada99b472ac 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -100,14 +100,13 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { logInfo("Stopped JobScheduler") } - def runJobs(time: Time, jobs: Seq[Job]) { - if (jobs.isEmpty) { - logInfo("No jobs added for time " + time) + def submitJobSet(jobSet: JobSet) { + if (jobSet.jobs.isEmpty) { + logInfo("No jobs added for time " + jobSet.time) } else { - val jobSet = new JobSet(time, jobs) - jobSets.put(time, jobSet) + jobSets.put(jobSet.time, jobSet) jobSet.jobs.foreach(job => jobExecutor.execute(new JobHandler(job))) - logInfo("Added jobs for time " + time) + logInfo("Added jobs for time " + jobSet.time) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala index fcf303aee6cd7..a69d74362173e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala @@ -24,7 +24,11 @@ import org.apache.spark.streaming.Time * belong to the same batch. */ private[streaming] -case class JobSet(time: Time, jobs: Seq[Job]) { +case class JobSet( + time: Time, + jobs: Seq[Job], + receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]] = Map.empty + ) { private val incompleteJobs = new HashSet[Job]() private val submissionTime = System.currentTimeMillis() // when this jobset was submitted @@ -60,6 +64,7 @@ case class JobSet(time: Time, jobs: Seq[Job]) { def toBatchInfo: BatchInfo = { new BatchInfo( time, + receivedBlockInfo, submissionTime, if (processingStartTime >= 0 ) Some(processingStartTime) else None, if (processingEndTime >= 0 ) Some(processingEndTime) else None diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala index 067e804202236..a1e6f5176825a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala @@ -17,20 +17,42 @@ package org.apache.spark.streaming.scheduler -import scala.collection.mutable.{HashMap, Queue, SynchronizedMap} +import scala.collection.mutable.{HashMap, SynchronizedMap, SynchronizedQueue} import akka.actor._ + import org.apache.spark.{Logging, SparkEnv, SparkException} import org.apache.spark.SparkContext._ -import org.apache.spark.storage.BlockId +import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.dstream.{NetworkReceiver, StopReceiver} import org.apache.spark.util.AkkaUtils +/** Information about receiver */ +case class ReceiverInfo(streamId: Int, typ: String, location: String) { + override def toString = s"$typ-$streamId" +} + +/** Information about blocks received by the network receiver */ +case class ReceivedBlockInfo( + streamId: Int, + blockId: StreamBlockId, + numRecords: Long, + metadata: Any + ) + +/** + * Messages used by the NetworkReceiver and the NetworkInputTracker to communicate + * with each other. + */ private[streaming] sealed trait NetworkInputTrackerMessage -private[streaming] case class RegisterReceiver(streamId: Int, receiverActor: ActorRef) - extends NetworkInputTrackerMessage -private[streaming] case class AddBlocks(streamId: Int, blockIds: Seq[BlockId], metadata: Any) +private[streaming] case class RegisterReceiver( + streamId: Int, + typ: String, + host: String, + receiverActor: ActorRef + ) extends NetworkInputTrackerMessage +private[streaming] case class AddBlock(receivedBlockInfo: ReceivedBlockInfo) extends NetworkInputTrackerMessage private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) extends NetworkInputTrackerMessage @@ -47,9 +69,10 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { val networkInputStreamMap = Map(networkInputStreams.map(x => (x.id, x)): _*) val receiverExecutor = new ReceiverExecutor() val receiverInfo = new HashMap[Int, ActorRef] with SynchronizedMap[Int, ActorRef] - val receivedBlockIds = new HashMap[Int, Queue[BlockId]] with SynchronizedMap[Int, Queue[BlockId]] + val receivedBlockInfo = new HashMap[Int, SynchronizedQueue[ReceivedBlockInfo]] + with SynchronizedMap[Int, SynchronizedQueue[ReceivedBlockInfo]] val timeout = AkkaUtils.askTimeout(ssc.conf) - + val listenerBus = ssc.scheduler.listenerBus // actor is created when generator starts. // This not being null means the tracker has been started and not stopped @@ -83,12 +106,32 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { } } + /** Return all the blocks received from a receiver. */ + def getReceivedBlockInfo(streamId: Int): Array[ReceivedBlockInfo] = { + val receivedBlockInfo = getReceivedBlockInfoQueue(streamId).dequeueAll(x => true) + logInfo("Stream " + streamId + " received " + receivedBlockInfo.size + " blocks") + receivedBlockInfo.toArray + } + + private def getReceivedBlockInfoQueue(streamId: Int) = { + receivedBlockInfo.getOrElseUpdate(streamId, new SynchronizedQueue[ReceivedBlockInfo]) + } + /** Register a receiver */ - def registerReceiver(streamId: Int, receiverActor: ActorRef, sender: ActorRef) { + def registerReceiver( + streamId: Int, + typ: String, + host: String, + receiverActor: ActorRef, + sender: ActorRef + ) { if (!networkInputStreamMap.contains(streamId)) { throw new Exception("Register received for unexpected id " + streamId) } receiverInfo += ((streamId, receiverActor)) + ssc.scheduler.listenerBus.post(StreamingListenerReceiverStarted( + ReceiverInfo(streamId, typ, host) + )) logInfo("Registered receiver for network stream " + streamId + " from " + sender.path.address) } @@ -98,35 +141,26 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { logError("Deregistered receiver for network stream " + streamId + " with message:\n" + message) } - /** Get all the received blocks for the given stream. */ - def getBlocks(streamId: Int, time: Time): Array[BlockId] = { - val queue = receivedBlockIds.getOrElseUpdate(streamId, new Queue[BlockId]()) - val result = queue.dequeueAll(x => true).toArray - logInfo("Stream " + streamId + " received " + result.size + " blocks") - result - } - /** Add new blocks for the given stream */ - def addBlocks(streamId: Int, blockIds: Seq[BlockId], metadata: Any) = { - val queue = receivedBlockIds.getOrElseUpdate(streamId, new Queue[BlockId]) - queue ++= blockIds - networkInputStreamMap(streamId).addMetadata(metadata) - logDebug("Stream " + streamId + " received new blocks: " + blockIds.mkString("[", ", ", "]")) + def addBlocks(receivedBlockInfo: ReceivedBlockInfo) { + getReceivedBlockInfoQueue(receivedBlockInfo.streamId) += receivedBlockInfo + logDebug("Stream " + receivedBlockInfo.streamId + " received new blocks: " + + receivedBlockInfo.blockId) } /** Check if any blocks are left to be processed */ def hasMoreReceivedBlockIds: Boolean = { - !receivedBlockIds.forall(_._2.isEmpty) + !receivedBlockInfo.values.forall(_.isEmpty) } /** Actor to receive messages from the receivers. */ private class NetworkInputTrackerActor extends Actor { def receive = { - case RegisterReceiver(streamId, receiverActor) => - registerReceiver(streamId, receiverActor, sender) + case RegisterReceiver(streamId, typ, host, receiverActor) => + registerReceiver(streamId, typ, host, receiverActor, sender) sender ! true - case AddBlocks(streamId, blockIds, metadata) => - addBlocks(streamId, blockIds, metadata) + case AddBlock(receivedBlockInfo) => + addBlocks(receivedBlockInfo) case DeregisterReceiver(streamId, message) => deregisterReceiver(streamId, message) sender ! true diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala index 461ea3506477f..5db40ebbeb1de 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala @@ -23,8 +23,11 @@ import org.apache.spark.util.Distribution /** Base trait for events related to StreamingListener */ sealed trait StreamingListenerEvent +case class StreamingListenerBatchSubmitted(batchInfo: BatchInfo) extends StreamingListenerEvent case class StreamingListenerBatchCompleted(batchInfo: BatchInfo) extends StreamingListenerEvent case class StreamingListenerBatchStarted(batchInfo: BatchInfo) extends StreamingListenerEvent +case class StreamingListenerReceiverStarted(receiverInfo: ReceiverInfo) + extends StreamingListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ private[scheduler] case object StreamingListenerShutdown extends StreamingListenerEvent @@ -34,14 +37,17 @@ private[scheduler] case object StreamingListenerShutdown extends StreamingListen * computation. */ trait StreamingListener { - /** - * Called when processing of a batch has completed - */ + + /** Called when a receiver has been started */ + def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { } + + /** Called when a batch of jobs has been submitted for processing. */ + def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) { } + + /** Called when processing of a batch of jobs has completed. */ def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { } - /** - * Called when processing of a batch has started - */ + /** Called when processing of a batch of jobs has started. */ def onBatchStarted(batchStarted: StreamingListenerBatchStarted) { } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala index 18811fc2b01d8..ea03dfc7bfeea 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala @@ -38,6 +38,10 @@ private[spark] class StreamingListenerBus() extends Logging { while (true) { val event = eventQueue.take event match { + case receiverStarted: StreamingListenerReceiverStarted => + listeners.foreach(_.onReceiverStarted(receiverStarted)) + case batchSubmitted: StreamingListenerBatchSubmitted => + listeners.foreach(_.onBatchSubmitted(batchSubmitted)) case batchStarted: StreamingListenerBatchStarted => listeners.foreach(_.onBatchStarted(batchStarted)) case batchCompleted: StreamingListenerBatchCompleted => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala new file mode 100644 index 0000000000000..8b025b09ed34d --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.ui + +import org.apache.spark.streaming.{Time, StreamingContext} +import org.apache.spark.streaming.scheduler._ +import scala.collection.mutable.{Queue, HashMap} +import org.apache.spark.streaming.scheduler.StreamingListenerReceiverStarted +import org.apache.spark.streaming.scheduler.StreamingListenerBatchStarted +import org.apache.spark.streaming.scheduler.BatchInfo +import org.apache.spark.streaming.scheduler.ReceiverInfo +import org.apache.spark.streaming.scheduler.StreamingListenerBatchSubmitted +import org.apache.spark.util.Distribution + + +private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends StreamingListener { + + private val waitingBatchInfos = new HashMap[Time, BatchInfo] + private val runningBatchInfos = new HashMap[Time, BatchInfo] + private val completedaBatchInfos = new Queue[BatchInfo] + private val batchInfoLimit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100) + private var totalCompletedBatches = 0L + private val receiverInfos = new HashMap[Int, ReceiverInfo] + + val batchDuration = ssc.graph.batchDuration.milliseconds + + override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) = { + synchronized { + receiverInfos.put(receiverStarted.receiverInfo.streamId, receiverStarted.receiverInfo) + } + } + + override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) = synchronized { + runningBatchInfos(batchSubmitted.batchInfo.batchTime) = batchSubmitted.batchInfo + } + + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted) = synchronized { + runningBatchInfos(batchStarted.batchInfo.batchTime) = batchStarted.batchInfo + waitingBatchInfos.remove(batchStarted.batchInfo.batchTime) + } + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) = synchronized { + waitingBatchInfos.remove(batchCompleted.batchInfo.batchTime) + runningBatchInfos.remove(batchCompleted.batchInfo.batchTime) + completedaBatchInfos.enqueue(batchCompleted.batchInfo) + if (completedaBatchInfos.size > batchInfoLimit) completedaBatchInfos.dequeue() + totalCompletedBatches += 1L + } + + def numNetworkReceivers = synchronized { + ssc.graph.getNetworkInputStreams().size + } + + def numTotalCompletedBatches: Long = synchronized { + totalCompletedBatches + } + + def numUnprocessedBatches: Long = synchronized { + waitingBatchInfos.size + runningBatchInfos.size + } + + def waitingBatches: Seq[BatchInfo] = synchronized { + waitingBatchInfos.values.toSeq + } + + def runningBatches: Seq[BatchInfo] = synchronized { + runningBatchInfos.values.toSeq + } + + def retainedCompletedBatches: Seq[BatchInfo] = synchronized { + completedaBatchInfos.toSeq + } + + def processingDelayDistribution: Option[Distribution] = synchronized { + extractDistribution(_.processingDelay) + } + + def schedulingDelayDistribution: Option[Distribution] = synchronized { + extractDistribution(_.schedulingDelay) + } + + def totalDelayDistribution: Option[Distribution] = synchronized { + extractDistribution(_.totalDelay) + } + + def receivedRecordsDistributions: Map[Int, Option[Distribution]] = synchronized { + val latestBatchInfos = retainedBatches.reverse.take(batchInfoLimit) + val latestBlockInfos = latestBatchInfos.map(_.receivedBlockInfo) + (0 until numNetworkReceivers).map { receiverId => + val blockInfoOfParticularReceiver = latestBlockInfos.map { batchInfo => + batchInfo.get(receiverId).getOrElse(Array.empty) + } + val recordsOfParticularReceiver = blockInfoOfParticularReceiver.map { blockInfo => + // calculate records per second for each batch + blockInfo.map(_.numRecords).sum.toDouble * 1000 / batchDuration + } + val distributionOption = Distribution(recordsOfParticularReceiver) + (receiverId, distributionOption) + }.toMap + } + + def lastReceivedBatchRecords: Map[Int, Long] = { + val lastReceivedBlockInfoOption = lastReceivedBatch.map(_.receivedBlockInfo) + lastReceivedBlockInfoOption.map { lastReceivedBlockInfo => + (0 until numNetworkReceivers).map { receiverId => + (receiverId, lastReceivedBlockInfo(receiverId).map(_.numRecords).sum) + }.toMap + }.getOrElse { + (0 until numNetworkReceivers).map(receiverId => (receiverId, 0L)).toMap + } + } + + def receiverInfo(receiverId: Int): Option[ReceiverInfo] = { + receiverInfos.get(receiverId) + } + + def lastCompletedBatch: Option[BatchInfo] = { + completedaBatchInfos.sortBy(_.batchTime)(Time.ordering).lastOption + } + + def lastReceivedBatch: Option[BatchInfo] = { + retainedBatches.lastOption + } + + private def retainedBatches: Seq[BatchInfo] = synchronized { + (waitingBatchInfos.values.toSeq ++ + runningBatchInfos.values.toSeq ++ completedaBatchInfos).sortBy(_.batchTime)(Time.ordering) + } + + private def extractDistribution(getMetric: BatchInfo => Option[Long]): Option[Distribution] = { + Distribution(completedaBatchInfos.flatMap(getMetric(_)).map(_.toDouble)) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala new file mode 100644 index 0000000000000..6607437db560a --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.ui + +import java.util.Calendar +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.Logging +import org.apache.spark.ui._ +import org.apache.spark.ui.UIUtils._ +import org.apache.spark.util.Distribution + +/** Page for Spark Web UI that shows statistics of a streaming job */ +private[ui] class StreamingPage(parent: StreamingTab) + extends WebUIPage("") with Logging { + + private val listener = parent.listener + private val startTime = Calendar.getInstance().getTime() + private val emptyCell = "-" + + /** Render the page */ + def render(request: HttpServletRequest): Seq[Node] = { + val content = + generateBasicStats() ++

      ++ +

      Statistics over last {listener.retainedCompletedBatches.size} processed batches

      ++ + generateNetworkStatsTable() ++ + generateBatchStatsTable() + UIUtils.headerSparkPage( + content, parent.basePath, parent.appName, "Streaming", parent.headerTabs, parent, Some(5000)) + } + + /** Generate basic stats of the streaming program */ + private def generateBasicStats(): Seq[Node] = { + val timeSinceStart = System.currentTimeMillis() - startTime.getTime +
        +
      • + Started at: {startTime.toString} +
      • +
      • + Time since start: {formatDurationVerbose(timeSinceStart)} +
      • +
      • + Network receivers: {listener.numNetworkReceivers} +
      • +
      • + Batch interval: {formatDurationVerbose(listener.batchDuration)} +
      • +
      • + Processed batches: {listener.numTotalCompletedBatches} +
      • +
      • + Waiting batches: {listener.numUnprocessedBatches} +
      • +
      + } + + /** Generate stats of data received over the network the streaming program */ + private def generateNetworkStatsTable(): Seq[Node] = { + val receivedRecordDistributions = listener.receivedRecordsDistributions + val lastBatchReceivedRecord = listener.lastReceivedBatchRecords + val table = if (receivedRecordDistributions.size > 0) { + val headerRow = Seq( + "Receiver", + "Location", + "Records in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]", + "Minimum rate\n[records/sec]", + "25th percentile rate\n[records/sec]", + "Median rate\n[records/sec]", + "75th percentile rate\n[records/sec]", + "Maximum rate\n[records/sec]" + ) + val dataRows = (0 until listener.numNetworkReceivers).map { receiverId => + val receiverInfo = listener.receiverInfo(receiverId) + val receiverName = receiverInfo.map(_.toString).getOrElse(s"Receiver-$receiverId") + val receiverLocation = receiverInfo.map(_.location).getOrElse(emptyCell) + val receiverLastBatchRecords = formatDurationVerbose(lastBatchReceivedRecord(receiverId)) + val receivedRecordStats = receivedRecordDistributions(receiverId).map { d => + d.getQuantiles().map(r => formatDurationVerbose(r.toLong)) + }.getOrElse { + Seq(emptyCell, emptyCell, emptyCell, emptyCell, emptyCell) + } + Seq(receiverName, receiverLocation, receiverLastBatchRecords) ++ receivedRecordStats + } + Some(listingTable(headerRow, dataRows)) + } else { + None + } + + val content = +
      Network Input Statistics
      ++ +
      {table.getOrElse("No network receivers")}
      + + content + } + + /** Generate stats of batch jobs of the streaming program */ + private def generateBatchStatsTable(): Seq[Node] = { + val numBatches = listener.retainedCompletedBatches.size + val lastCompletedBatch = listener.lastCompletedBatch + val table = if (numBatches > 0) { + val processingDelayQuantilesRow = { + Seq( + "Processing Time", + formatDurationOption(lastCompletedBatch.flatMap(_.processingDelay)) + ) ++ getQuantiles(listener.processingDelayDistribution) + } + val schedulingDelayQuantilesRow = { + Seq( + "Scheduling Delay", + formatDurationOption(lastCompletedBatch.flatMap(_.schedulingDelay)) + ) ++ getQuantiles(listener.schedulingDelayDistribution) + } + val totalDelayQuantilesRow = { + Seq( + "Total Delay", + formatDurationOption(lastCompletedBatch.flatMap(_.totalDelay)) + ) ++ getQuantiles(listener.totalDelayDistribution) + } + val headerRow = Seq("Metric", "Last batch", "Minimum", "25th percentile", + "Median", "75th percentile", "Maximum") + val dataRows: Seq[Seq[String]] = Seq( + processingDelayQuantilesRow, + schedulingDelayQuantilesRow, + totalDelayQuantilesRow + ) + Some(listingTable(headerRow, dataRows)) + } else { + None + } + + val content = +
      Batch Processing Statistics
      ++ +
      +
        + {table.getOrElse("No statistics have been generated yet.")} +
      +
      + + content + } + + + /** + * Returns a human-readable string representing a duration such as "5 second 35 ms" + */ + private def formatDurationOption(msOption: Option[Long]): String = { + msOption.map(formatDurationVerbose).getOrElse(emptyCell) + } + + /** Get quantiles for any time distribution */ + private def getQuantiles(timeDistributionOption: Option[Distribution]) = { + timeDistributionOption.get.getQuantiles().map { ms => formatDurationVerbose(ms.toLong) } + } + + /** Generate HTML table from string data */ + private def listingTable(headers: Seq[String], data: Seq[Seq[String]]) = { + def generateDataRow(data: Seq[String]): Seq[Node] = { + {data.map(d => {d})} + } + UIUtils.listingTable(headers, generateDataRow, data, fixedWidth = true) + } +} + diff --git a/core/src/test/scala/org/apache/spark/SparkUISuite.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala similarity index 58% rename from core/src/test/scala/org/apache/spark/SparkUISuite.scala rename to streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala index d0d119c15081d..51448d15c6516 100644 --- a/core/src/test/scala/org/apache/spark/SparkUISuite.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -15,21 +15,22 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.streaming.ui -import java.net.URI +import org.apache.spark.Logging +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.ui.WebUITab -import org.scalatest.FunSuite +/** Spark Web UI tab that shows statistics of a streaming job */ +private[spark] class StreamingTab(ssc: StreamingContext) + extends WebUITab(ssc.sc.ui, "streaming") with Logging { -class SparkUISuite extends FunSuite with SharedSparkContext { + val parent = ssc.sc.ui + val appName = parent.appName + val basePath = parent.basePath + val listener = new StreamingJobProgressListener(ssc) - test("verify appUIAddress contains the scheme") { - val uiAddress = sc.ui.appUIAddress - assert(uiAddress.equals("http://" + sc.ui.appUIHostPort)) - } - - test("verify appUIAddress contains the port") { - val splitUIAddress = sc.ui.appUIAddress.split(':') - assert(splitUIAddress(2).toInt == sc.ui.boundPort) - } + ssc.addStreamingListener(listener) + attachPage(new StreamingPage(this)) + parent.attachTab(this) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 389b23d4d5e4b..952511d411a8e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -239,11 +239,11 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { /** This is a server to test the network input stream */ -class TestServer() extends Logging { +class TestServer(portToBind: Int = 0) extends Logging { val queue = new ArrayBlockingQueue[String](100) - val serverSocket = new ServerSocket(0) + val serverSocket = new ServerSocket(portToBind) val servingThread = new Thread() { override def run() { @@ -282,7 +282,7 @@ class TestServer() extends Logging { def start() { servingThread.start() } - def send(msg: String) { queue.add(msg) } + def send(msg: String) { queue.put(msg) } def stop() { servingThread.interrupt() } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 9cc27ef7f03b5..efd0d22ecb57a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -161,7 +161,6 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w } } - test("stop only streaming context") { ssc = new StreamingContext(master, appName, batchDuration) sc = ssc.sparkContext diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala new file mode 100644 index 0000000000000..35538ec188f67 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +import scala.io.Source + +import org.scalatest.FunSuite +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ + +class UISuite extends FunSuite { + + test("streaming tab in spark UI") { + val ssc = new StreamingContext("local", "test", Seconds(1)) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(ssc.sparkContext.ui.appUIAddress).mkString + assert(!html.contains("random data that should not be present")) + // test if streaming tab exist + assert(html.toLowerCase.contains("streaming")) + // test if other Spark tabs still exist + assert(html.toLowerCase.contains("stages")) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL( + ssc.sparkContext.ui.appUIAddress.stripSuffix("/") + "/streaming").mkString + assert(html.toLowerCase.contains("batch")) + assert(html.toLowerCase.contains("network")) + } + } +} From c2d160fbee2ef90a7683d9771f2f632b68d74aef Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 12 Apr 2014 16:33:38 -0700 Subject: [PATCH 063/641] [Fix #204] Update out-dated comments This PR is self-explanatory. Author: Andrew Or Closes #381 from andrewor14/master and squashes the following commits: 3e8dde2 [Andrew Or] Fix comments for #204 --- core/src/main/scala/org/apache/spark/SparkContext.scala | 6 +----- .../org/apache/spark/scheduler/ReplayListenerBus.scala | 4 ++-- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a764c174d562c..5a36e6f5c19a9 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1191,11 +1191,7 @@ class SparkContext(config: SparkConf) extends Logging { listenerBus.post(SparkListenerApplicationStart(appName, startTime, sparkUser)) } - /** - * Post the application end event to all listeners immediately, rather than adding it - * to the event queue for it to be asynchronously processed eventually. Otherwise, a race - * condition exists in which the listeners may stop before this event has been propagated. - */ + /** Post the application end event */ private def postApplicationEnd() { listenerBus.post(SparkListenerApplicationEnd(System.currentTimeMillis)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index f868e772cf58a..f89724d4ea196 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -31,8 +31,8 @@ import org.apache.spark.util.JsonProtocol /** * A SparkListenerBus that replays logged events from persisted storage. * - * This class expects files to be appropriately prefixed as specified in EventLoggingListener. - * There exists a one-to-one mapping between ReplayListenerBus and event logging applications. + * This assumes the given paths are valid log files, where each line can be deserialized into + * exactly one SparkListenerEvent. */ private[spark] class ReplayListenerBus( logPaths: Seq[Path], From ca11919e6e97a62eb3e3ce882ffa29eae36f50f7 Mon Sep 17 00:00:00 2001 From: Bharath Bhushan Date: Sat, 12 Apr 2014 20:52:29 -0700 Subject: [PATCH 064/641] [SPARK-1403] Move the class loader creation back to where it was in 0.9.0 [SPARK-1403] I investigated why spark 0.9.0 loads fine on mesos while spark 1.0.0 fails. What I found was that in SparkEnv.scala, while creating the SparkEnv object, the current thread's classloader is null. But in 0.9.0, at the same place, it is set to org.apache.spark.repl.ExecutorClassLoader . I saw that https://github.com/apache/spark/commit/7edbea41b43e0dc11a2de156be220db8b7952d01 moved it to it current place. I moved it back and saw that 1.0.0 started working fine on mesos. I just created a minimal patch that allows me to run spark on mesos correctly. It seems like SecurityManager's creation needs to be taken into account for a correct fix. Also moving the creation of the serializer out of SparkEnv might be a part of the right solution. PTAL. Author: Bharath Bhushan Closes #322 from manku-timma/spark-1403 and squashes the following commits: 606c2b9 [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 ec8f870 [Bharath Bhushan] revert the logger change for java 6 compatibility as PR 334 is doing it 728beca [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 044027d [Bharath Bhushan] fix compile error 6f260a4 [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 b3a053f [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 04b9662 [Bharath Bhushan] add missing line 4803c19 [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 f3c9a14 [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 42d3d6a [Bharath Bhushan] used code fragment from @ueshin to fix the problem in a better way 89109d7 [Bharath Bhushan] move the class loader creation back to where it was in 0.9.0 --- .../spark/executor/MesosExecutorBackend.scala | 22 +++++++++++++------ 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 6fc702fdb1512..df36a06485c77 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -50,13 +50,21 @@ private[spark] class MesosExecutorBackend executorInfo: ExecutorInfo, frameworkInfo: FrameworkInfo, slaveInfo: SlaveInfo) { - logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) - this.driver = driver - val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) - executor = new Executor( - executorInfo.getExecutorId.getValue, - slaveInfo.getHostname, - properties) + val cl = Thread.currentThread.getContextClassLoader + try { + // Work around for SPARK-1480 + Thread.currentThread.setContextClassLoader(getClass.getClassLoader) + logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) + this.driver = driver + val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) + executor = new Executor( + executorInfo.getExecutorId.getValue, + slaveInfo.getHostname, + properties) + } finally { + // Work around for SPARK-1480 + Thread.currentThread.setContextClassLoader(cl) + } } override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) { From 4bc07eebbf5e2ea0c0b6f1642049515025d88d07 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 13 Apr 2014 08:58:37 -0700 Subject: [PATCH 065/641] SPARK-1480: Clean up use of classloaders The Spark codebase is a bit fast-and-loose when accessing classloaders and this has caused a few bugs to surface in master. This patch defines some utility methods for accessing classloaders. This makes the intention when accessing a classloader much more explicit in the code and fixes a few cases where the wrong one was chosen. case (a) -> We want the classloader that loaded Spark case (b) -> We want the context class loader, or if not present, we want (a) This patch provides a better fix for SPARK-1403 (https://issues.apache.org/jira/browse/SPARK-1403) than the current work around, which it reverts. It also fixes a previously unreported bug that the `./spark-submit` script did not work for running with `local` master. It didn't work because the executor classloader did not properly delegate to the context class loader (if it is defined) and in local mode the context class loader is set by the `./spark-submit` script. A unit test is added for that case. Author: Patrick Wendell Closes #398 from pwendell/class-loaders and squashes the following commits: b4a1a58 [Patrick Wendell] Minor clean up 14f1272 [Patrick Wendell] SPARK-1480: Clean up use of classloaders --- .../main/scala/org/apache/spark/Logging.scala | 4 +-- .../org/apache/spark/executor/Executor.scala | 6 ++-- .../spark/executor/MesosExecutorBackend.scala | 22 ++++--------- .../apache/spark/metrics/MetricsConfig.scala | 3 +- .../apache/spark/scheduler/ResultTask.scala | 1 - .../spark/scheduler/SchedulableBuilder.scala | 3 +- .../spark/scheduler/TaskResultGetter.scala | 4 +-- .../spark/serializer/JavaSerializer.scala | 3 +- .../org/apache/spark/ui/JettyUtils.scala | 3 +- .../scala/org/apache/spark/util/Utils.scala | 15 +++++++++ .../ExecutorURLClassLoaderSuite.scala | 32 +++++++++++++++++-- .../org/apache/spark/repl/SparkILoop.scala | 7 ++-- .../spark/sql/catalyst/util/package.scala | 4 ++- .../compression/compressionSchemes.scala | 3 +- .../sql/execution/SparkSqlSerializer.scala | 3 +- 15 files changed, 78 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 9d429dceeb858..50d8e93e1f0d7 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -22,6 +22,7 @@ import org.slf4j.{Logger, LoggerFactory} import org.slf4j.impl.StaticLoggerBinder import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.Utils /** * :: DeveloperApi :: @@ -115,8 +116,7 @@ trait Logging { val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements if (!log4jInitialized && usingLog4j) { val defaultLogProps = "org/apache/spark/log4j-defaults.properties" - val classLoader = this.getClass.getClassLoader - Option(classLoader.getResource(defaultLogProps)) match { + Option(Utils.getSparkClassLoader.getResource(defaultLogProps)) match { case Some(url) => PropertyConfigurator.configure(url) log.info(s"Using Spark's default log4j profile: $defaultLogProps") 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 c12bd922d40e4..f89b2bffd1676 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -292,7 +292,7 @@ private[spark] class Executor( * created by the interpreter to the search path */ private def createClassLoader(): MutableURLClassLoader = { - val loader = this.getClass.getClassLoader + val currentLoader = Utils.getContextOrSparkClassLoader // For each of the jars in the jarSet, add them to the class loader. // We assume each of the files has already been fetched. @@ -301,8 +301,8 @@ private[spark] class Executor( }.toArray val userClassPathFirst = conf.getBoolean("spark.files.userClassPathFirst", false) userClassPathFirst match { - case true => new ChildExecutorURLClassLoader(urls, loader) - case false => new ExecutorURLClassLoader(urls, loader) + case true => new ChildExecutorURLClassLoader(urls, currentLoader) + case false => new ExecutorURLClassLoader(urls, currentLoader) } } diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index df36a06485c77..6fc702fdb1512 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -50,21 +50,13 @@ private[spark] class MesosExecutorBackend executorInfo: ExecutorInfo, frameworkInfo: FrameworkInfo, slaveInfo: SlaveInfo) { - val cl = Thread.currentThread.getContextClassLoader - try { - // Work around for SPARK-1480 - Thread.currentThread.setContextClassLoader(getClass.getClassLoader) - logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) - this.driver = driver - val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) - executor = new Executor( - executorInfo.getExecutorId.getValue, - slaveInfo.getHostname, - properties) - } finally { - // Work around for SPARK-1480 - Thread.currentThread.setContextClassLoader(cl) - } + logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) + this.driver = driver + val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) + executor = new Executor( + executorInfo.getExecutorId.getValue, + slaveInfo.getHostname, + properties) } override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) { diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index 3e3e18c3537d0..1b7a5d1f1980a 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -24,6 +24,7 @@ import scala.collection.mutable import scala.util.matching.Regex import org.apache.spark.Logging +import org.apache.spark.util.Utils private[spark] class MetricsConfig(val configFile: Option[String]) extends Logging { @@ -50,7 +51,7 @@ private[spark] class MetricsConfig(val configFile: Option[String]) extends Loggi try { is = configFile match { case Some(f) => new FileInputStream(f) - case None => getClass.getClassLoader.getResourceAsStream(METRICS_CONF) + case None => Utils.getSparkClassLoader.getResourceAsStream(METRICS_CONF) } if (is != null) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 083fb895d8696..0b381308b61ff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -54,7 +54,6 @@ private[spark] object ResultTask { def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = { - val loader = Thread.currentThread.getContextClassLoader val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) val ser = SparkEnv.get.closureSerializer.newInstance() val objIn = ser.deserializeStream(in) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index e4eced383c3a5..6c5827f75e636 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -23,6 +23,7 @@ import java.util.{NoSuchElementException, Properties} import scala.xml.XML import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.util.Utils /** * An interface to build Schedulable tree @@ -72,7 +73,7 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) schedulerAllocFile.map { f => new FileInputStream(f) }.getOrElse { - getClass.getClassLoader.getResourceAsStream(DEFAULT_SCHEDULER_FILE) + Utils.getSparkClassLoader.getResourceAsStream(DEFAULT_SCHEDULER_FILE) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index cb4ad4ae9350c..c9ad2b151daf0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -85,13 +85,13 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul try { if (serializedData != null && serializedData.limit() > 0) { reason = serializer.get().deserialize[TaskEndReason]( - serializedData, getClass.getClassLoader) + serializedData, Utils.getSparkClassLoader) } } catch { case cnd: ClassNotFoundException => // Log an error but keep going here -- the task failed, so not catastropic if we can't // deserialize the reason. - val loader = Thread.currentThread.getContextClassLoader + val loader = Utils.getContextOrSparkClassLoader logError( "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) case ex: Throwable => {} 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 5e5883554fcc1..e9163deaf2036 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -23,6 +23,7 @@ import java.nio.ByteBuffer import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.ByteBufferInputStream +import org.apache.spark.util.Utils private[spark] class JavaSerializationStream(out: OutputStream, counterReset: Int) extends SerializationStream { @@ -86,7 +87,7 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } def deserializeStream(s: InputStream): DeserializationStream = { - new JavaDeserializationStream(s, Thread.currentThread.getContextClassLoader) + new JavaDeserializationStream(s, Utils.getContextOrSparkClassLoader) } def deserializeStream(s: InputStream, loader: ClassLoader): DeserializationStream = { 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 62a4e3d0f6a42..3ae147a36c8a4 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -33,6 +33,7 @@ import org.json4s.JValue import org.json4s.jackson.JsonMethods.{pretty, render} import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.util.Utils /** * Utilities for launching a web server using Jetty's HTTP Server class @@ -124,7 +125,7 @@ private[spark] object JettyUtils extends Logging { contextHandler.setInitParameter("org.eclipse.jetty.servlet.Default.gzip", "false") val staticHandler = new DefaultServlet val holder = new ServletHolder(staticHandler) - Option(getClass.getClassLoader.getResource(resourceBase)) match { + Option(Utils.getSparkClassLoader.getResource(resourceBase)) match { case Some(res) => holder.setInitParameter("resourceBase", res.toString) case None => diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 166f48ce7342e..a3af4e7b91692 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -116,6 +116,21 @@ private[spark] object Utils extends Logging { } } + /** + * Get the ClassLoader which loaded Spark. + */ + def getSparkClassLoader = getClass.getClassLoader + + /** + * Get the Context ClassLoader on this thread or, if not present, the ClassLoader that + * loaded Spark. + * + * This should be used whenever passing a ClassLoader to Class.ForName or finding the currently + * active loader when setting up ClassLoader delegation chains. + */ + def getContextOrSparkClassLoader = + Option(Thread.currentThread().getContextClassLoader).getOrElse(getSparkClassLoader) + /** * Primitive often used when writing {@link java.nio.ByteBuffer} to {@link java.io.DataOutput}. */ diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala index c40cfc0696fce..e2050e95a1b88 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala @@ -17,12 +17,12 @@ package org.apache.spark.executor -import java.io.File import java.net.URLClassLoader import org.scalatest.FunSuite -import org.apache.spark.TestUtils +import org.apache.spark.{LocalSparkContext, SparkContext, SparkException, TestUtils} +import org.apache.spark.util.Utils class ExecutorURLClassLoaderSuite extends FunSuite { @@ -63,5 +63,33 @@ class ExecutorURLClassLoaderSuite extends FunSuite { } } + test("driver sets context class loader in local mode") { + // Test the case where the driver program sets a context classloader and then runs a job + // in local mode. This is what happens when ./spark-submit is called with "local" as the + // master. + val original = Thread.currentThread().getContextClassLoader + val className = "ClassForDriverTest" + val jar = TestUtils.createJarWithClasses(Seq(className)) + val contextLoader = new URLClassLoader(Array(jar), Utils.getContextOrSparkClassLoader) + Thread.currentThread().setContextClassLoader(contextLoader) + + val sc = new SparkContext("local", "driverLoaderTest") + + try { + sc.makeRDD(1 to 5, 2).mapPartitions { x => + val loader = Thread.currentThread().getContextClassLoader + Class.forName(className, true, loader).newInstance() + Seq().iterator + }.count() + } + catch { + case e: SparkException if e.getMessage.contains("ClassNotFoundException") => + fail("Local executor could not find class", e) + case t: Throwable => fail("Unexpected exception ", t) + } + + sc.stop() + Thread.currentThread().setContextClassLoader(original) + } } diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 5a367b6bb79de..beb40e87024bd 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -39,6 +39,7 @@ import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse} import org.apache.spark.Logging import org.apache.spark.SparkConf import org.apache.spark.SparkContext +import org.apache.spark.util.Utils /** The Scala interactive shell. It provides a read-eval-print loop * around the Interpreter class. @@ -130,7 +131,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, def history = in.history /** The context class loader at the time this object was created */ - protected val originalClassLoader = Thread.currentThread.getContextClassLoader + protected val originalClassLoader = Utils.getContextOrSparkClassLoader // classpath entries added via :cp var addedClasspath: String = "" @@ -177,7 +178,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, override lazy val formatting = new Formatting { def prompt = SparkILoop.this.prompt } - override protected def parentClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse(classOf[SparkILoop].getClassLoader) + override protected def parentClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse(classOf[SparkILoop].getClassLoader) } /** Create a new interpreter. */ @@ -871,7 +872,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } val u: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe - val m = u.runtimeMirror(getClass.getClassLoader) + val m = u.runtimeMirror(Utils.getSparkClassLoader) private def tagOfStaticClass[T: ClassTag]: u.TypeTag[T] = u.TypeTag[T]( m, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index a001d953592db..49fc4f70fdfae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst import java.io.{PrintWriter, ByteArrayOutputStream, FileInputStream, File} +import org.apache.spark.util.{Utils => SparkUtils} + package object util { /** * Returns a path to a temporary file that probably does not exist. @@ -54,7 +56,7 @@ package object util { def resourceToString( resource:String, encoding: String = "UTF-8", - classLoader: ClassLoader = this.getClass.getClassLoader) = { + classLoader: ClassLoader = SparkUtils.getSparkClassLoader) = { val inStream = classLoader.getResourceAsStream(resource) val outStream = new ByteArrayOutputStream try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index df8220b556edd..e92cf5ac4f9df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -26,6 +26,7 @@ import scala.reflect.runtime.universe.runtimeMirror import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar._ +import org.apache.spark.util.Utils private[sql] case object PassThrough extends CompressionScheme { override val typeId = 0 @@ -254,7 +255,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { private val dictionary = { // TODO Can we clean up this mess? Maybe move this to `DataType`? implicit val classTag = { - val mirror = runtimeMirror(getClass.getClassLoader) + val mirror = runtimeMirror(Utils.getSparkClassLoader) ClassTag[T#JvmType](mirror.runtimeClass(columnType.scalaTag.tpe)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index d8e1b970c1d88..c30ae5bcc02d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -25,6 +25,7 @@ import com.esotericsoftware.kryo.{Serializer, Kryo} import org.apache.spark.{SparkEnv, SparkConf} import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.MutablePair +import org.apache.spark.util.Utils class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { override def newKryo(): Kryo = { @@ -44,7 +45,7 @@ class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { kryo.register(classOf[scala.collection.mutable.ArrayBuffer[_]]) kryo.register(classOf[scala.math.BigDecimal], new BigDecimalSerializer) kryo.setReferences(false) - kryo.setClassLoader(this.getClass.getClassLoader) + kryo.setClassLoader(Utils.getSparkClassLoader) kryo } } From 037fe4d2ba01be5610baa3dd9c5c9d3a5e5e1064 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Sun, 13 Apr 2014 13:18:52 -0700 Subject: [PATCH 066/641] [SPARK-1415] Hadoop min split for wholeTextFiles() JIRA issue [here](https://issues.apache.org/jira/browse/SPARK-1415). New Hadoop API of `InputFormat` does not provide the `minSplits` parameter, which makes the API incompatible between `HadoopRDD` and `NewHadoopRDD`. The PR is for constructing compatible APIs. Though `minSplits` is deprecated by New Hadoop API, we think it is better to make APIs compatible here. **Note** that `minSplits` in `wholeTextFiles` could only be treated as a *suggestion*, the real number of splits may not be greater than `minSplits` due to `isSplitable()=false`. Author: Xusen Yin Closes #376 from yinxusen/hadoop-min-split and squashes the following commits: 76417f6 [Xusen Yin] refine comments c10af60 [Xusen Yin] refine comments and rewrite new class for wholeTextFile 766d05b [Xusen Yin] refine Java API and comments 4875755 [Xusen Yin] add minSplits for WholeTextFiles --- .../scala/org/apache/spark/SparkContext.scala | 17 ++++-- .../spark/api/java/JavaSparkContext.scala | 14 ++++- .../input/WholeTextFileInputFormat.scala | 14 +++++ .../org/apache/spark/rdd/NewHadoopRDD.scala | 60 +++++++++++++++---- .../java/org/apache/spark/JavaAPISuite.java | 2 +- .../WholeTextFileRecordReaderSuite.scala | 2 +- 6 files changed, 90 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5a36e6f5c19a9..456070fa7c5ef 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -454,14 +454,21 @@ class SparkContext(config: SparkConf) extends Logging { * (a-hdfs-path/part-nnnnn, its content) * }}} * - * @note Small files are preferred, as each file will be loaded fully in memory. + * @note Small files are preferred, large file is also allowable, but may cause bad performance. + * + * @param minSplits A suggestion value of the minimal splitting number for input data. */ - def wholeTextFiles(path: String): RDD[(String, String)] = { - newAPIHadoopFile( - path, + def wholeTextFiles(path: String, minSplits: Int = defaultMinSplits): RDD[(String, String)] = { + val job = new NewHadoopJob(hadoopConfiguration) + NewFileInputFormat.addInputPath(job, new Path(path)) + val updateConf = job.getConfiguration + new WholeTextFileRDD( + this, classOf[WholeTextFileInputFormat], classOf[String], - classOf[String]) + classOf[String], + updateConf, + minSplits) } /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 1e8242a2cbbce..7fbefe1cb0fb1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -177,7 +177,19 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * (a-hdfs-path/part-nnnnn, its content) * }}} * - * @note Small files are preferred, as each file will be loaded fully in memory. + * @note Small files are preferred, large file is also allowable, but may cause bad performance. + * + * @param minSplits A suggestion value of the minimal splitting number for input data. + */ + def wholeTextFiles(path: String, minSplits: Int): JavaPairRDD[String, String] = + new JavaPairRDD(sc.wholeTextFiles(path, minSplits)) + + /** + * Read a directory of text files from HDFS, a local file system (available on all nodes), or any + * Hadoop-supported file system URI. Each file is read as a single record and returned in a + * key-value pair, where the key is the path of each file, the value is the content of each file. + * + * @see `wholeTextFiles(path: String, minSplits: Int)`. */ def wholeTextFiles(path: String): JavaPairRDD[String, String] = new JavaPairRDD(sc.wholeTextFiles(path)) diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala index 4887fb6b84eb2..80d055a89573b 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala @@ -17,6 +17,8 @@ package org.apache.spark.input +import scala.collection.JavaConversions._ + import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.InputSplit import org.apache.hadoop.mapreduce.JobContext @@ -44,4 +46,16 @@ private[spark] class WholeTextFileInputFormat extends CombineFileInputFormat[Str context, classOf[WholeTextFileRecordReader]) } + + /** + * Allow minSplits set by end-user in order to keep compatibility with old Hadoop API. + */ + def setMaxSplitSize(context: JobContext, minSplits: Int) { + val files = listStatus(context) + val totalLen = files.map { file => + if (file.isDir) 0L else file.getLen + }.sum + val maxSplitSize = Math.ceil(totalLen * 1.0 / (if (minSplits == 0) 1 else minSplits)).toLong + super.setMaxSplitSize(maxSplitSize) + } } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 2d8dfa5a1645a..8684b645bc361 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -24,11 +24,18 @@ import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ -import org.apache.spark.{InterruptibleIterator, Logging, Partition, SerializableWritable, SparkContext, TaskContext} import org.apache.spark.annotation.DeveloperApi - -private[spark] -class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable) +import org.apache.spark.input.WholeTextFileInputFormat +import org.apache.spark.InterruptibleIterator +import org.apache.spark.Logging +import org.apache.spark.Partition +import org.apache.spark.SerializableWritable +import org.apache.spark.{SparkContext, TaskContext} + +private[spark] class NewHadoopPartition( + rddId: Int, + val index: Int, + @transient rawSplit: InputSplit with Writable) extends Partition { val serializableHadoopSplit = new SerializableWritable(rawSplit) @@ -65,17 +72,19 @@ class NewHadoopRDD[K, V]( private val confBroadcast = sc.broadcast(new SerializableWritable(conf)) // private val serializableConf = new SerializableWritable(conf) - private val jobtrackerId: String = { + private val jobTrackerId: String = { val formatter = new SimpleDateFormat("yyyyMMddHHmm") formatter.format(new Date()) } - @transient private val jobId = new JobID(jobtrackerId, id) + @transient protected val jobId = new JobID(jobTrackerId, id) override def getPartitions: Array[Partition] = { val inputFormat = inputFormatClass.newInstance - if (inputFormat.isInstanceOf[Configurable]) { - inputFormat.asInstanceOf[Configurable].setConf(conf) + inputFormat match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => } val jobContext = newJobContext(conf, jobId) val rawSplits = inputFormat.getSplits(jobContext).toArray @@ -91,11 +100,13 @@ class NewHadoopRDD[K, V]( val split = theSplit.asInstanceOf[NewHadoopPartition] logInfo("Input split: " + split.serializableHadoopSplit) val conf = confBroadcast.value.value - val attemptId = newTaskAttemptID(jobtrackerId, id, isMap = true, split.index, 0) + val attemptId = newTaskAttemptID(jobTrackerId, id, isMap = true, split.index, 0) val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) val format = inputFormatClass.newInstance - if (format.isInstanceOf[Configurable]) { - format.asInstanceOf[Configurable].setConf(conf) + format match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => } val reader = format.createRecordReader( split.serializableHadoopSplit.value, hadoopAttemptContext) @@ -141,3 +152,30 @@ class NewHadoopRDD[K, V]( def getConf: Configuration = confBroadcast.value.value } +private[spark] class WholeTextFileRDD( + sc : SparkContext, + inputFormatClass: Class[_ <: WholeTextFileInputFormat], + keyClass: Class[String], + valueClass: Class[String], + @transient conf: Configuration, + minSplits: Int) + extends NewHadoopRDD[String, String](sc, inputFormatClass, keyClass, valueClass, conf) { + + override def getPartitions: Array[Partition] = { + val inputFormat = inputFormatClass.newInstance + inputFormat match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => + } + val jobContext = newJobContext(conf, jobId) + inputFormat.setMaxSplitSize(jobContext, minSplits) + val rawSplits = inputFormat.getSplits(jobContext).toArray + val result = new Array[Partition](rawSplits.size) + for (i <- 0 until rawSplits.size) { + result(i) = new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) + } + result + } +} + diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index ab2fdac553349..8d2e9f1846343 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -626,7 +626,7 @@ public void wholeTextFiles() throws IOException { container.put(tempDirName+"/part-00000", new Text(content1).toString()); container.put(tempDirName+"/part-00001", new Text(content2).toString()); - JavaPairRDD readRDD = sc.wholeTextFiles(tempDirName); + JavaPairRDD readRDD = sc.wholeTextFiles(tempDirName, 3); List> result = readRDD.collect(); for (Tuple2 res : result) { diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index e89b296d41026..33d6de9a76405 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -73,7 +73,7 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { createNativeFile(dir, filename, contents) } - val res = sc.wholeTextFiles(dir.toString).collect() + val res = sc.wholeTextFiles(dir.toString, 3).collect() assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size, "Number of files read out does not fit with the actual value.") From 7dbca68e92416ec5f023c8807bb06470c01a6d3a Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 14 Apr 2014 15:22:43 -0700 Subject: [PATCH 067/641] [BUGFIX] In-memory columnar storage bug fixes Fixed several bugs of in-memory columnar storage to make `HiveInMemoryCompatibilitySuite` pass. @rxin @marmbrus It is reasonable to include `HiveInMemoryCompatibilitySuite` in this PR, but I didn't, since it significantly increases test execution time. What do you think? **UPDATE** `HiveCompatibilitySuite` has been made to cache tables in memory. `HiveInMemoryCompatibilitySuite` was removed. Author: Cheng Lian Author: Michael Armbrust Closes #374 from liancheng/inMemBugFix and squashes the following commits: 6ad6d9b [Cheng Lian] Merged HiveCompatibilitySuite and HiveInMemoryCompatibilitySuite 5bdbfe7 [Cheng Lian] Revert 882c538 & 8426ddc, which introduced regression 882c538 [Cheng Lian] Remove attributes field from InMemoryColumnarTableScan 32cc9ce [Cheng Lian] Code style cleanup 99382bf [Cheng Lian] Enable compression by default 4390bcc [Cheng Lian] Report error for any Throwable in HiveComparisonTest d1df4fd [Michael Armbrust] Remove test tables that might always get created anyway? ab9e807 [Michael Armbrust] Fix the logged console version of failed test cases to use the new syntax. 1965123 [Michael Armbrust] Don't use coalesce for gathering all data to a single partition, as it does not work correctly with mutable rows. e36cdd0 [Michael Armbrust] Spelling. 2d0e168 [Michael Armbrust] Run Hive tests in-memory too. 6360723 [Cheng Lian] Made PreInsertionCasts support SparkLogicalPlan and InMemoryColumnarTableScan c9b0f6f [Cheng Lian] Let InsertIntoTable support InMemoryColumnarTableScan 9c8fc40 [Cheng Lian] Disable compression by default e619995 [Cheng Lian] Bug fix: incorrect byte order in CompressionScheme.columnHeaderSize 8426ddc [Cheng Lian] Bug fix: InMemoryColumnarTableScan should cache columns specified by the attributes argument 036cd09 [Cheng Lian] Clean up unused imports 44591a5 [Cheng Lian] Bug fix: NullableColumnAccessor.hasNext must take nulls into account 052bf41 [Cheng Lian] Bug fix: should only gather compressibility info for non-null values 95b3301 [Cheng Lian] Fixed bugs in IntegralDelta --- .../org/apache/spark/sql/SchemaRDD.scala | 2 +- .../sql/columnar/NullableColumnAccessor.scala | 2 + .../CompressibleColumnBuilder.scala | 4 +- .../compression/CompressionScheme.scala | 4 +- .../compression/compressionSchemes.scala | 20 ++++----- .../apache/spark/sql/execution/Exchange.scala | 9 +++- .../spark/sql/execution/SparkPlan.scala | 4 +- .../apache/spark/sql/CachedTableSuite.scala | 5 +-- ...scala => InMemoryColumnarQuerySuite.scala} | 12 +++++- .../NullableColumnAccessorSuite.scala | 4 ++ .../compression/IntegralDeltaSuite.scala | 15 +++++-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 42 ++++++++++++------- .../spark/sql/hive/HiveStrategies.scala | 4 ++ .../org/apache/spark/sql/hive/TestHive.scala | 10 ++--- .../org/apache/spark/sql/hive/hiveUdfs.scala | 16 +++---- .../hive/execution/HiveComparisonTest.scala | 10 +---- .../execution/HiveCompatibilitySuite.scala | 12 +++++- 17 files changed, 109 insertions(+), 66 deletions(-) rename sql/core/src/test/scala/org/apache/spark/sql/columnar/{ColumnarQuerySuite.scala => InMemoryColumnarQuerySuite.scala} (79%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 16da7fd92bffe..91500416eefaa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -99,7 +99,7 @@ class SchemaRDD( def baseSchemaRDD = this // ========================================================================================= - // RDD functions: Copy the interal row representation so we present immutable data to users. + // RDD functions: Copy the internal row representation so we present immutable data to users. // ========================================================================================= override def compute(split: Partition, context: TaskContext): Iterator[Row] = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala index 7d49ab07f7a53..b7f8826861a2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala @@ -54,4 +54,6 @@ private[sql] trait NullableColumnAccessor extends ColumnAccessor { pos += 1 } + + abstract override def hasNext = seenNulls < nullCount || super.hasNext } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index fd3b1adf9687a..0f808f68f2eec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -65,7 +65,9 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] abstract override def appendFrom(row: Row, ordinal: Int) { super.appendFrom(row, ordinal) - gatherCompressibilityStats(row, ordinal) + if (!row.isNullAt(ordinal)) { + gatherCompressibilityStats(row, ordinal) + } } abstract override def build() = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala index c605a8e4434e3..ba1810dd2ae66 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.columnar.compression -import java.nio.ByteBuffer +import java.nio.{ByteOrder, ByteBuffer} import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnType, NativeColumnType} @@ -84,7 +84,7 @@ private[sql] object CompressionScheme { } def columnHeaderSize(columnBuffer: ByteBuffer): Int = { - val header = columnBuffer.duplicate() + val header = columnBuffer.duplicate().order(ByteOrder.nativeOrder) val nullCount = header.getInt(4) // Column type ID + null count + null positions 4 + 4 + 4 * nullCount diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index e92cf5ac4f9df..800009d3195e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -397,26 +397,27 @@ private[sql] sealed abstract class IntegralDelta[I <: IntegralType] extends Comp if (initial) { initial = false - prev = value _compressedSize += 1 + columnType.defaultSize } else { val (smallEnough, _) = byteSizedDelta(value, prev) _compressedSize += (if (smallEnough) 1 else 1 + columnType.defaultSize) } + + prev = value } override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[I]) = { to.putInt(typeId) if (from.hasRemaining) { - val prev = columnType.extract(from) - + var prev = columnType.extract(from) to.put(Byte.MinValue) columnType.append(prev, to) while (from.hasRemaining) { val current = columnType.extract(from) val (smallEnough, delta) = byteSizedDelta(current, prev) + prev = current if (smallEnough) { to.put(delta) @@ -443,13 +444,8 @@ private[sql] sealed abstract class IntegralDelta[I <: IntegralType] extends Comp override def next() = { val delta = buffer.get() - - if (delta > Byte.MinValue) { - addDelta(prev, delta) - } else { - prev = columnType.extract(buffer) - prev - } + prev = if (delta > Byte.MinValue) addDelta(prev, delta) else columnType.extract(buffer) + prev } override def hasNext = buffer.hasRemaining @@ -465,7 +461,7 @@ private[sql] case object IntDelta extends IntegralDelta[IntegerType.type] { override protected def byteSizedDelta(x: Int, y: Int): (Boolean, Byte) = { val delta = x - y - if (delta < Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) + if (math.abs(delta) <= Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) } } @@ -478,6 +474,6 @@ private[sql] case object LongDelta extends IntegralDelta[LongType.type] { override protected def byteSizedDelta(x: Long, y: Long): (Boolean, Byte) = { val delta = x - y - if (delta < Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) + if (math.abs(delta) <= Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 450c142c0baa4..070557e47c4c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -61,7 +61,14 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una shuffled.map(_._1) case SinglePartition => - child.execute().coalesce(1, shuffle = true) + val rdd = child.execute().mapPartitions { iter => + val mutablePair = new MutablePair[Null, Row]() + iter.map(r => mutablePair.update(null, r)) + } + val partitioner = new HashPartitioner(1) + val shuffled = new ShuffledRDD[Null, Row, MutablePair[Null, Row]](rdd, partitioner) + shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + shuffled.map(_._2) case _ => sys.error(s"Exchange not implemented for $newPartitioning") // TODO: Handle BroadcastPartitioning. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index daa423cb8ea1a..5d89697db5f99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -70,8 +70,8 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan) SparkLogicalPlan( alreadyPlanned match { case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) - case InMemoryColumnarTableScan(output, child) => - InMemoryColumnarTableScan(output.map(_.newInstance), child) + case scan @ InMemoryColumnarTableScan(output, child) => + scan.copy(attributes = output.map(_.newInstance)) case _ => sys.error("Multiple instance of the same relation detected.") }).asInstanceOf[this.type] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 7c6a642278226..0331f90272a99 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -17,11 +17,10 @@ package org.apache.spark.sql -import org.scalatest.FunSuite import org.apache.spark.sql.TestData._ -import org.apache.spark.sql.test.TestSQLContext -import org.apache.spark.sql.execution.SparkLogicalPlan import org.apache.spark.sql.columnar.InMemoryColumnarTableScan +import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.test.TestSQLContext class CachedTableSuite extends QueryTest { TestData // Load test tables. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala similarity index 79% rename from sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 2ed4cf2170f9d..16a13b8a74960 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -18,10 +18,11 @@ package org.apache.spark.sql.columnar import org.apache.spark.sql.{QueryTest, TestData} +import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.execution.SparkLogicalPlan import org.apache.spark.sql.test.TestSQLContext -class ColumnarQuerySuite extends QueryTest { +class InMemoryColumnarQuerySuite extends QueryTest { import TestData._ import TestSQLContext._ @@ -32,6 +33,15 @@ class ColumnarQuerySuite extends QueryTest { checkAnswer(scan, testData.collect().toSeq) } + test("projection") { + val plan = TestSQLContext.executePlan(testData.select('value, 'key).logicalPlan).executedPlan + val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan)) + + checkAnswer(scan, testData.collect().map { + case Row(key: Int, value: String) => value -> key + }.toSeq) + } + test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 4a21eb6201a69..35ab14cbc353d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -68,12 +68,16 @@ class NullableColumnAccessorSuite extends FunSuite { val row = new GenericMutableRow(1) (0 until 4).foreach { _ => + assert(accessor.hasNext) accessor.extractTo(row, 0) assert(row(0) === randomRow(0)) + assert(accessor.hasNext) accessor.extractTo(row, 0) assert(row.isNullAt(0)) } + + assert(!accessor.hasNext) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala index 1390e5eef6106..ce419ca7269ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types.IntegralType import org.apache.spark.sql.columnar._ +import org.apache.spark.sql.columnar.ColumnarTestUtils._ class IntegralDeltaSuite extends FunSuite { testIntegralDelta(new IntColumnStats, INT, IntDelta) @@ -63,7 +64,7 @@ class IntegralDeltaSuite extends FunSuite { } else { val oneBoolean = columnType.defaultSize 1 + oneBoolean + deltas.map { - d => if (math.abs(d) < Byte.MaxValue) 1 else 1 + oneBoolean + d => if (math.abs(d) <= Byte.MaxValue) 1 else 1 + oneBoolean }.sum }) @@ -78,7 +79,7 @@ class IntegralDeltaSuite extends FunSuite { expectResult(input.head, "The first value is wrong")(columnType.extract(buffer)) (input.tail, deltas).zipped.foreach { (value, delta) => - if (delta < Byte.MaxValue) { + if (math.abs(delta) <= Byte.MaxValue) { expectResult(delta, "Wrong delta")(buffer.get()) } else { expectResult(Byte.MinValue, "Expecting escaping mark here")(buffer.get()) @@ -105,11 +106,17 @@ class IntegralDeltaSuite extends FunSuite { test(s"$scheme: simple case") { val input = columnType match { - case INT => Seq(1: Int, 2: Int, 130: Int) - case LONG => Seq(1: Long, 2: Long, 130: Long) + case INT => Seq(2: Int, 1: Int, 2: Int, 130: Int) + case LONG => Seq(2: Long, 1: Long, 2: Long, 130: Long) } skeleton(input.map(_.asInstanceOf[I#JvmType])) } + + test(s"$scheme: long random series") { + // Have to workaround with `Any` since no `ClassTag[I#JvmType]` available here. + val input = Array.fill[Any](10000)(makeRandomValue(columnType)) + skeleton(input.map(_.asInstanceOf[I#JvmType])) + } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index fc053c56c052d..c36b5878cb007 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -33,6 +33,8 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.columnar.InMemoryColumnarTableScan /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -115,23 +117,31 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { case p: LogicalPlan if !p.childrenResolved => p case p @ InsertIntoTable(table: MetastoreRelation, _, child, _) => - val childOutputDataTypes = child.output.map(_.dataType) - // Only check attributes, not partitionKeys since they are always strings. - // TODO: Fully support inserting into partitioned tables. - val tableOutputDataTypes = table.attributes.map(_.dataType) - - if (childOutputDataTypes == tableOutputDataTypes) { - p - } else { - // Only do the casting when child output data types differ from table output data types. - val castedChildOutput = child.output.zip(table.output).map { - case (input, output) if input.dataType != output.dataType => - Alias(Cast(input, output.dataType), input.name)() - case (input, _) => input - } - - p.copy(child = logical.Project(castedChildOutput, child)) + castChildOutput(p, table, child) + + case p @ logical.InsertIntoTable(SparkLogicalPlan(InMemoryColumnarTableScan( + _, HiveTableScan(_, table, _))), _, child, _) => + castChildOutput(p, table, child) + } + + def castChildOutput(p: InsertIntoTable, table: MetastoreRelation, child: LogicalPlan) = { + val childOutputDataTypes = child.output.map(_.dataType) + // Only check attributes, not partitionKeys since they are always strings. + // TODO: Fully support inserting into partitioned tables. + val tableOutputDataTypes = table.attributes.map(_.dataType) + + if (childOutputDataTypes == tableOutputDataTypes) { + p + } else { + // Only do the casting when child output data types differ from table output data types. + val castedChildOutput = child.output.zip(table.output).map { + case (input, output) if input.dataType != output.dataType => + Alias(Cast(input, output.dataType), input.name)() + case (input, _) => input } + + p.copy(child = logical.Project(castedChildOutput, child)) + } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 3ca1d93c11fa9..ac817b21a152e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution._ +import org.apache.spark.sql.columnar.InMemoryColumnarTableScan trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. @@ -42,6 +43,9 @@ trait HiveStrategies { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil + case logical.InsertIntoTable(SparkLogicalPlan(InMemoryColumnarTableScan( + _, HiveTableScan(_, table, _))), partition, child, overwrite) => + InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 2fea9702954d7..465e5f146fe71 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -160,12 +160,6 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { TestTable("src1", "CREATE TABLE src1 (key INT, value STRING)".cmd, s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), - TestTable("dest1", - "CREATE TABLE IF NOT EXISTS dest1 (key INT, value STRING)".cmd), - TestTable("dest2", - "CREATE TABLE IF NOT EXISTS dest2 (key INT, value STRING)".cmd), - TestTable("dest3", - "CREATE TABLE IF NOT EXISTS dest3 (key INT, value STRING)".cmd), TestTable("srcpart", () => { runSqlHive( "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)") @@ -257,6 +251,7 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { private val loadedTables = new collection.mutable.HashSet[String] + var cacheTables: Boolean = false def loadTestTable(name: String) { if (!(loadedTables contains name)) { // Marks the table as loaded first to prevent infite mutually recursive table loading. @@ -265,6 +260,9 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) createCmds.foreach(_()) + + if (cacheTables) + cacheTable(name) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index f9b437d435eba..55a4363af6c76 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -130,8 +130,7 @@ trait HiveFunctionFactory { } } -abstract class HiveUdf - extends Expression with Logging with HiveFunctionFactory { +abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory { self: Product => type UDFType @@ -146,7 +145,7 @@ abstract class HiveUdf lazy val functionInfo = getFunctionInfo(name) lazy val function = createFunction[UDFType](name) - override def toString = s"${nodeName}#${functionInfo.getDisplayName}(${children.mkString(",")})" + override def toString = s"$nodeName#${functionInfo.getDisplayName}(${children.mkString(",")})" } case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUdf { @@ -202,10 +201,11 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd } } -case class HiveGenericUdf( - name: String, - children: Seq[Expression]) extends HiveUdf with HiveInspectors { +case class HiveGenericUdf(name: String, children: Seq[Expression]) + extends HiveUdf with HiveInspectors { + import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ + type UDFType = GenericUDF @transient @@ -357,7 +357,7 @@ case class HiveGenericUdaf( override def toString = s"$nodeName#$name(${children.mkString(",")})" - def newInstance = new HiveUdafFunction(name, children, this) + def newInstance() = new HiveUdafFunction(name, children, this) } /** @@ -435,7 +435,7 @@ case class HiveGenericUdtf( } } - override def toString() = s"$nodeName#$name(${children.mkString(",")})" + override def toString = s"$nodeName#$name(${children.mkString(",")})" } case class HiveUdafFunction( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 3cc4562a88d66..6c91f40d0f925 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -218,10 +218,7 @@ abstract class HiveComparisonTest val quotes = "\"\"\"" queryList.zipWithIndex.map { case (query, i) => - s""" - |val q$i = $quotes$query$quotes.q - |q$i.stringResult() - """.stripMargin + s"""val q$i = hql($quotes$query$quotes); q$i.collect()""" }.mkString("\n== Console version of this test ==\n", "\n", "\n") } @@ -287,7 +284,6 @@ abstract class HiveComparisonTest |Error: ${e.getMessage} |${stackTraceToString(e)} |$queryString - |$consoleTestCase """.stripMargin stringToFile( new File(hiveFailedDirectory, testCaseName), @@ -304,7 +300,7 @@ abstract class HiveComparisonTest val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => val query = new TestHive.HiveQLQueryExecution(queryString) try { (query, prepareAnswer(query, query.stringResult())) } catch { - case e: Exception => + case e: Throwable => val errorMessage = s""" |Failed to execute query using catalyst: @@ -313,8 +309,6 @@ abstract class HiveComparisonTest |$query |== HIVE - ${hive.size} row(s) == |${hive.mkString("\n")} - | - |$consoleTestCase """.stripMargin stringToFile(new File(failedDirectory, testCaseName), errorMessage + consoleTestCase) fail(errorMessage) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index f76e16bc1afc5..c3cfa3d25a5c2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -17,16 +17,26 @@ package org.apache.spark.sql.hive.execution +import org.scalatest.BeforeAndAfter + import org.apache.spark.sql.hive.TestHive /** * Runs the test cases that are included in the hive distribution. */ -class HiveCompatibilitySuite extends HiveQueryFileTest { +class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // TODO: bundle in jar files... get from classpath lazy val hiveQueryDir = TestHive.getHiveFile("ql/src/test/queries/clientpositive") def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) + override def beforeAll() { + TestHive.cacheTables = true + } + + override def afterAll() { + TestHive.cacheTables = false + } + /** A list of tests deemed out of scope currently and thus completely disregarded. */ override def blackList = Seq( // These tests use hooks that are not on the classpath and thus break all subsequent execution. From 268b53567c93538c03cb66276ed9e05c9f1d3ac6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 14 Apr 2014 15:51:54 -0700 Subject: [PATCH 068/641] HOTFIX: Use file name and not paths for excludes --- .rat-excludes | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.rat-excludes b/.rat-excludes index 8954330bd10a7..9e9abb3f10bbf 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -40,5 +40,5 @@ work golden test.out/* .*iml -python/metastore/service.properties -python/metastore/db.lck +service.properties +db.lck From 0247b5c5467ca1b0d03ba929a78fa4d805582d84 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 14 Apr 2014 19:50:00 -0700 Subject: [PATCH 069/641] SPARK-1488. Resolve scalac feature warnings during build For your consideration: scalac currently notes a number of feature warnings during compilation: ``` [warn] there were 65 feature warning(s); re-run with -feature for details ``` Warnings are like: ``` [warn] /Users/srowen/Documents/spark/core/src/main/scala/org/apache/spark/SparkContext.scala:1261: implicit conversion method rddToPairRDDFunctions should be enabled [warn] by making the implicit value scala.language.implicitConversions visible. [warn] This can be achieved by adding the import clause 'import scala.language.implicitConversions' [warn] or by setting the compiler option -language:implicitConversions. [warn] See the Scala docs for value scala.language.implicitConversions for a discussion [warn] why the feature should be explicitly enabled. [warn] implicit def rddToPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = [warn] ^ ``` scalac is suggesting that it's just best practice to explicitly enable certain language features by importing them where used. This PR simply adds the imports it suggests (and squashes one other Java warning along the way). This leaves just deprecation warnings in the build. Author: Sean Owen Closes #404 from srowen/SPARK-1488 and squashes the following commits: 8598980 [Sean Owen] Quiet scalac warnings about language features by explicitly importing language features. 39bc831 [Sean Owen] Enable -feature in scalac to emit language feature warnings --- bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala | 2 ++ core/src/main/scala/org/apache/spark/SparkContext.scala | 1 + .../main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala | 1 + .../src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala | 1 + core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala | 1 + .../scala/org/apache/spark/api/java/JavaSparkContext.scala | 1 + .../scala/org/apache/spark/deploy/FaultToleranceTest.scala | 1 + .../src/main/scala/org/apache/spark/deploy/master/Master.scala | 1 + .../src/main/scala/org/apache/spark/deploy/worker/Worker.scala | 1 + .../scala/org/apache/spark/network/ConnectionManager.scala | 1 + core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala | 1 + core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala | 1 + .../scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala | 1 + .../src/main/scala/org/apache/spark/scheduler/ResultTask.scala | 1 + .../main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala | 1 + .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 1 + core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 1 + .../org/apache/spark/util/TimeStampedWeakValueHashMap.scala | 1 + core/src/main/scala/org/apache/spark/util/Vector.scala | 1 + .../test/scala/org/apache/spark/ConnectionManagerSuite.scala | 2 +- core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala | 2 ++ core/src/test/scala/org/apache/spark/DriverSuite.scala | 2 ++ core/src/test/scala/org/apache/spark/PipedRDDSuite.scala | 3 +-- .../test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala | 1 + .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 1 + .../scala/org/apache/spark/storage/BlockManagerSuite.scala | 3 +++ .../scala/org/apache/spark/storage/DiskBlockManagerSuite.scala | 1 + core/src/test/scala/org/apache/spark/ui/UISuite.scala | 1 + .../org/apache/spark/util/random/XORShiftRandomSuite.scala | 2 ++ .../org/apache/spark/streaming/examples/ZeroMQWordCount.scala | 2 ++ graphx/src/main/scala/org/apache/spark/graphx/Graph.scala | 1 + .../org/apache/spark/graphx/impl/MessageToPartition.scala | 1 + .../main/scala/org/apache/spark/graphx/impl/Serializers.scala | 3 ++- .../scala/org/apache/spark/graphx/util/BytecodeUtils.scala | 1 + .../main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala | 1 + .../scala/org/apache/spark/mllib/util/MFDataGenerator.scala | 1 + .../java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java | 1 + pom.xml | 1 + project/SparkBuild.scala | 2 +- .../main/scala/org/apache/spark/sql/catalyst/SqlParser.scala | 1 + .../scala/org/apache/spark/streaming/StreamingContext.scala | 1 + .../org/apache/spark/streaming/api/java/JavaDStream.scala | 1 + .../org/apache/spark/streaming/api/java/JavaDStreamLike.scala | 1 + .../org/apache/spark/streaming/api/java/JavaPairDStream.scala | 1 + .../org/apache/spark/streaming/receivers/ActorReceiver.scala | 1 + .../apache/spark/streaming/scheduler/NetworkInputTracker.scala | 1 + .../org/apache/spark/streaming/StreamingContextSuite.scala | 2 ++ .../src/test/scala/org/apache/spark/streaming/UISuite.scala | 1 + .../org/apache/spark/tools/JavaAPICompletenessChecker.scala | 1 + 49 files changed, 58 insertions(+), 5 deletions(-) diff --git a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala index 69144e3e657bf..8e0f82ddb8897 100644 --- a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala @@ -24,6 +24,8 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.storage.StorageLevel +import scala.language.postfixOps + class TestVertex(val active: Boolean, val age: Int) extends Vertex with Serializable class TestMessage(val targetId: String) extends Message[String] with Serializable diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 456070fa7c5ef..3ddc0d5eeefb8 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -25,6 +25,7 @@ import java.util.UUID.randomUUID import scala.collection.{Map, Set} import scala.collection.generic.Growable import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.language.implicitConversions import scala.reflect.{ClassTag, classTag} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index 537f410b0ca26..4330cef3965ee 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -19,6 +19,7 @@ package org.apache.spark.api.java import java.lang.{Double => JDouble} +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.Partitioner diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index a41c7dbda2afc..e5b2c8a5e7cb1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -21,6 +21,7 @@ import java.util.{Comparator, List => JList} import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ +import scala.language.implicitConversions import scala.reflect.ClassTag import com.google.common.base.Optional diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 01d9357a2556d..327c1552dc941 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -17,6 +17,7 @@ package org.apache.spark.api.java +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark._ diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 7fbefe1cb0fb1..e6a3f06b0ea42 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -22,6 +22,7 @@ import java.util.{Map => JMap} import scala.collection.JavaConversions import scala.collection.JavaConversions._ +import scala.language.implicitConversions import scala.reflect.ClassTag import com.google.common.base.Optional diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index f4eb1601be3e4..47dbcd87c35b5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -25,6 +25,7 @@ import scala.collection.mutable.ListBuffer import scala.concurrent.{Await, future, promise} import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ +import scala.language.postfixOps import scala.sys.process._ import org.json4s._ diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 6c58e741df001..81f990bfa6513 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -23,6 +23,7 @@ import java.util.Date import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await import scala.concurrent.duration._ +import scala.language.postfixOps import scala.util.Random import akka.actor._ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 52c164ca3c574..dd0a1360abe14 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -23,6 +23,7 @@ import java.util.Date import scala.collection.mutable.HashMap import scala.concurrent.duration._ +import scala.language.postfixOps import akka.actor._ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index cfee41c61362e..dcbbc1853186b 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -33,6 +33,7 @@ import scala.collection.mutable.SynchronizedQueue import scala.concurrent.{Await, ExecutionContext, Future, Promise} import scala.concurrent.duration._ +import scala.language.postfixOps import org.apache.spark._ import org.apache.spark.util.{SystemClock, Utils} 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 c6e79557f08a1..9ff76892aed32 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer +import scala.language.existentials import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index 44401a663440c..c45b759f007cc 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -21,6 +21,7 @@ import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.language.existentials import scala.reflect.ClassTag import org.apache.spark._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 7367c08b5d324..0800c5684c60f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import java.util.Properties import scala.collection.mutable.Map +import scala.language.existentials import org.apache.spark._ import org.apache.spark.executor.TaskMetrics diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 0b381308b61ff..0e8d551e4b2ab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -21,6 +21,7 @@ import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashMap +import scala.language.existentials import org.apache.spark._ import org.apache.spark.rdd.{RDD, RDDCheckpointData} 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 23f3b3e824762..02b62de7e36b6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -21,6 +21,7 @@ import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashMap +import scala.language.existentials import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index acd152dda89d4..a3439b525fde1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -25,6 +25,7 @@ import scala.concurrent.duration._ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet +import scala.language.postfixOps import scala.util.Random import org.apache.spark._ 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 3ae147a36c8a4..750f5a501c213 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -22,6 +22,7 @@ import javax.servlet.DispatcherType import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.annotation.tailrec +import scala.language.implicitConversions import scala.util.{Failure, Success, Try} import scala.xml.Node diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala index b65017d6806c6..f5be5856c2109 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala @@ -21,6 +21,7 @@ import java.lang.ref.WeakReference import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable +import scala.language.implicitConversions import org.apache.spark.Logging diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index 1a647fa1c9d84..c6cab82c3e546 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -17,6 +17,7 @@ package org.apache.spark.util +import scala.language.implicitConversions import scala.util.Random import org.apache.spark.util.random.XORShiftRandom diff --git a/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala index 80f7ec00c74b2..df6b2604c8d8a 100644 --- a/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.network.{ConnectionManager, Message, ConnectionManagerId import scala.concurrent.Await import scala.concurrent.TimeoutException import scala.concurrent.duration._ - +import scala.language.postfixOps /** * Test the ConnectionManager with various security settings. diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index e50981cf6fb20..5a8310090890d 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark import java.lang.ref.WeakReference import scala.collection.mutable.{HashSet, SynchronizedSet} +import scala.language.existentials +import scala.language.postfixOps import scala.util.Random import org.scalatest.{BeforeAndAfter, FunSuite} diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 7f59bdcce4cc7..de4bd90c8f7e5 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -29,6 +29,8 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.util.Utils +import scala.language.postfixOps + class DriverSuite extends FunSuite with Timeouts { test("driver should exit after finishing") { diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index 867b28cc0d971..dfe057515efa0 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark import java.io.File -import com.google.common.io.Files - import org.scalatest.FunSuite import org.apache.spark.rdd.{HadoopRDD, PipedRDD, HadoopPartition} @@ -28,6 +26,7 @@ import org.apache.hadoop.mapred.{JobConf, TextInputFormat, FileSplit} import org.apache.hadoop.fs.Path import scala.collection.Map +import scala.language.postfixOps import scala.sys.process._ import scala.util.Try diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index 3b833f2e41867..28197657e9bad 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -22,6 +22,7 @@ import java.util.concurrent.Semaphore import scala.concurrent.{Await, TimeoutException} import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global +import scala.language.postfixOps import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Timeouts diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index db4df1d1212ff..35a7ac9d049c2 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import scala.Tuple2 import scala.collection.mutable.{HashSet, HashMap, Map} +import scala.language.reflectiveCalls import org.scalatest.{BeforeAndAfter, FunSuite} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index e10ec7d2624a0..907428db80af3 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -33,6 +33,9 @@ import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} +import scala.language.implicitConversions +import scala.language.postfixOps + class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { private val conf = new SparkConf(false) var store: BlockManager = null diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 808ddfdcf45d8..9b29e2a8a55de 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.storage import java.io.{File, FileWriter} import scala.collection.mutable +import scala.language.reflectiveCalls import com.google.common.io.Files import org.scalatest.{BeforeAndAfterEach, FunSuite} diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index b85c483ca2a08..ed02b0ba00d43 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -21,6 +21,7 @@ import java.net.ServerSocket import javax.servlet.http.HttpServletRequest import scala.io.Source +import scala.language.postfixOps import scala.util.{Failure, Success, Try} import org.eclipse.jetty.server.Server diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index 39199a1a17ccd..0865c6386f7cd 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -22,6 +22,8 @@ import org.scalatest.matchers.ShouldMatchers import org.apache.spark.util.Utils.times +import scala.language.reflectiveCalls + class XORShiftRandomSuite extends FunSuite with ShouldMatchers { def fixture = new { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index 445d2028582af..6f88db1abf19d 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -27,6 +27,8 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.zeromq._ +import scala.language.implicitConversions + /** * A simple publisher for demonstration purposes, repeatedly publishes random Messages * every one second. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index ef05623d7a0a1..45349692cbf6c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -17,6 +17,7 @@ package org.apache.spark.graphx +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.graphx.impl._ 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 9d4f3750cb8e4..c45ba3d2f8c24 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 @@ -17,6 +17,7 @@ package org.apache.spark.graphx.impl +import scala.language.implicitConversions import scala.reflect.{classTag, ClassTag} import org.apache.spark.Partitioner 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 2f2c524df6394..2f0531ee5f379 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 @@ -20,10 +20,11 @@ package org.apache.spark.graphx.impl import java.io.{EOFException, InputStream, OutputStream} import java.nio.ByteBuffer -import org.apache.spark.SparkConf import org.apache.spark.graphx._ import org.apache.spark.serializer._ +import scala.language.existentials + private[graphx] class VertexIdMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala index 087b1156f690b..be6b9047d932d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala @@ -20,6 +20,7 @@ package org.apache.spark.graphx.util import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.collection.mutable.HashSet +import scala.language.existentials import org.apache.spark.util.Utils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala index 873de871fd884..365b5e75d7f75 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.rdd +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index 3f413faca6bb4..b76fbe89c3681 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.util +import scala.language.postfixOps import scala.util.Random import org.jblas.DoubleMatrix diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java index c6d8425ffc38d..1421067dc61ed 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java @@ -36,6 +36,7 @@ public void denseArrayConstruction() { @Test public void sparseArrayConstruction() { + @SuppressWarnings("unchecked") Vector v = Vectors.sparse(3, Lists.>newArrayList( new Tuple2(0, 2.0), new Tuple2(2, 3.0))); diff --git a/pom.xml b/pom.xml index 5f66cbe768592..0eacedf7a6533 100644 --- a/pom.xml +++ b/pom.xml @@ -648,6 +648,7 @@ -unchecked -deprecation + -feature -Xms1024m diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a6058bba3d211..aac07b9f6129d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -162,7 +162,7 @@ object SparkBuild extends Build { organization := "org.apache.spark", version := SPARK_VERSION, scalaVersion := "2.10.4", - scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation", + scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation", "-feature", "-target:" + SCALAC_JVM_VERSION), javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 5b6aea81cb7d1..13a19d0adf5e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst +import scala.language.implicitConversions import scala.util.parsing.combinator.lexical.StdLexical import scala.util.parsing.combinator.syntactical.StandardTokenParsers import scala.util.parsing.input.CharArrayReader.EofCh diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index ff5d0aaa3d0bd..e9a4f7ba22576 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.Map import scala.collection.mutable.Queue +import scala.language.implicitConversions import scala.reflect.ClassTag import akka.actor.{Props, SupervisorStrategy} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala index 721d50273259e..13e2bacc92edc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala @@ -23,6 +23,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.storage.StorageLevel import org.apache.spark.rdd.RDD +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.streaming.dstream.DStream diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index bb2f492d06a00..a6184de4e83c1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -22,6 +22,7 @@ import java.lang.{Long => JLong} import java.util.{List => JList} import scala.collection.JavaConversions._ +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaRDDLike} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index 2ac943d7bf781..cb8e8f00a7b82 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -21,6 +21,7 @@ import java.lang.{Long => JLong, Iterable => JIterable} import java.util.{List => JList} import scala.collection.JavaConversions._ +import scala.language.implicitConversions import scala.reflect.ClassTag import com.google.common.base.Optional diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala index f5984d03c5342..da0d364ae7bdb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala @@ -23,6 +23,7 @@ import akka.actor.{ PossiblyHarmful, OneForOneStrategy } import akka.actor.SupervisorStrategy._ import scala.concurrent.duration._ +import scala.language.postfixOps import scala.reflect.ClassTag import org.apache.spark.storage.{StorageLevel, StreamBlockId} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala index a1e6f5176825a..438e72a7ced89 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming.scheduler import scala.collection.mutable.{HashMap, SynchronizedMap, SynchronizedQueue} +import scala.language.existentials import akka.actor._ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index efd0d22ecb57a..ad5367ab941a2 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -28,6 +28,8 @@ import org.scalatest.concurrent.Timeouts import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.SpanSugar._ +import scala.language.postfixOps + class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts with Logging { val master = "local[2]" diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala index 35538ec188f67..031e93ab24a70 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming import scala.io.Source +import scala.language.postfixOps import org.scalatest.FunSuite import org.scalatest.concurrent.Eventually._ diff --git a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala index 420522433e1e8..8d0f09933c8d3 100644 --- a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala +++ b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala @@ -20,6 +20,7 @@ package org.apache.spark.tools import java.lang.reflect.Method import scala.collection.mutable.ArrayBuffer +import scala.language.existentials import org.apache.spark._ import org.apache.spark.api.java._ From c99bcb7feaa761c5826f2e1d844d0502a3b79538 Mon Sep 17 00:00:00 2001 From: Ahir Reddy Date: Tue, 15 Apr 2014 00:07:55 -0700 Subject: [PATCH 070/641] SPARK-1374: PySpark API for SparkSQL An initial API that exposes SparkSQL functionality in PySpark. A PythonRDD composed of dictionaries, with string keys and primitive values (boolean, float, int, long, string) can be converted into a SchemaRDD that supports sql queries. ``` from pyspark.context import SQLContext sqlCtx = SQLContext(sc) rdd = sc.parallelize([{"field1" : 1, "field2" : "row1"}, {"field1" : 2, "field2": "row2"}, {"field1" : 3, "field2": "row3"}]) srdd = sqlCtx.applySchema(rdd) sqlCtx.registerRDDAsTable(srdd, "table1") srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") srdd2.collect() ``` The last line yields ```[{"f1" : 1, "f2" : "row1"}, {"f1" : 2, "f2": "row2"}, {"f1" : 3, "f2": "row3"}]``` Author: Ahir Reddy Author: Michael Armbrust Closes #363 from ahirreddy/pysql and squashes the following commits: 0294497 [Ahir Reddy] Updated log4j properties to supress Hive Warns 307d6e0 [Ahir Reddy] Style fix 6f7b8f6 [Ahir Reddy] Temporary fix MIMA checker. Since we now assemble Spark jar with Hive, we don't want to check the interfaces of all of our hive dependencies 3ef074a [Ahir Reddy] Updated documentation because classes moved to sql.py 29245bf [Ahir Reddy] Cache underlying SchemaRDD instead of generating and caching PythonRDD f2312c7 [Ahir Reddy] Moved everything into sql.py a19afe4 [Ahir Reddy] Doc fixes 6d658ba [Ahir Reddy] Remove the metastore directory created by the HiveContext tests in SparkSQL 521ff6d [Ahir Reddy] Trying to get spark to build with hive ab95eba [Ahir Reddy] Set SPARK_HIVE=true on jenkins ded03e7 [Ahir Reddy] Added doc test for HiveContext 22de1d4 [Ahir Reddy] Fixed maven pyrolite dependency e4da06c [Ahir Reddy] Display message if hive is not built into spark 227a0be [Michael Armbrust] Update API links. Fix Hive example. 58e2aa9 [Michael Armbrust] Build Docs for pyspark SQL Api. Minor fixes. 4285340 [Michael Armbrust] Fix building of Hive API Docs. 38a92b0 [Michael Armbrust] Add note to future non-python developers about python docs. 337b201 [Ahir Reddy] Changed com.clearspring.analytics stream version from 2.4.0 to 2.5.1 to match SBT build, and added pyrolite to maven build 40491c9 [Ahir Reddy] PR Changes + Method Visibility 1836944 [Michael Armbrust] Fix comments. e00980f [Michael Armbrust] First draft of python sql programming guide. b0192d3 [Ahir Reddy] Added Long, Double and Boolean as usable types + unit test f98a422 [Ahir Reddy] HiveContexts 79621cf [Ahir Reddy] cleaning up cruft b406ba0 [Ahir Reddy] doctest formatting 20936a5 [Ahir Reddy] Added tests and documentation e4d21b4 [Ahir Reddy] Added pyrolite dependency 79f739d [Ahir Reddy] added more tests 7515ba0 [Ahir Reddy] added more tests :) d26ec5e [Ahir Reddy] added test e9f5b8d [Ahir Reddy] adding tests 906d180 [Ahir Reddy] added todo explaining cost of creating Row object in python 251f99d [Ahir Reddy] for now only allow dictionaries as input 09b9980 [Ahir Reddy] made jrdd explicitly lazy c608947 [Ahir Reddy] SchemaRDD now has all RDD operations 725c91e [Ahir Reddy] awesome row objects 55d1c76 [Ahir Reddy] return row objects 4fe1319 [Ahir Reddy] output dictionaries correctly be079de [Ahir Reddy] returning dictionaries works cd5f79f [Ahir Reddy] Switched to using Scala SQLContext e948bd9 [Ahir Reddy] yippie 4886052 [Ahir Reddy] even better c0fb1c6 [Ahir Reddy] more working 043ca85 [Ahir Reddy] working 5496f9f [Ahir Reddy] doesn't crash b8b904b [Ahir Reddy] Added schema rdd class 67ba875 [Ahir Reddy] java to python, and python to java bcc0f23 [Ahir Reddy] Java to python ab6025d [Ahir Reddy] compiling --- core/pom.xml | 5 + .../apache/spark/api/python/PythonRDD.scala | 32 ++ dev/run-tests | 1 + docs/README.md | 2 +- docs/_plugins/copy_api_dirs.rb | 4 +- docs/sql-programming-guide.md | 103 ++++- pom.xml | 2 +- project/SparkBuild.scala | 3 +- python/pyspark/__init__.py | 18 +- python/pyspark/java_gateway.py | 4 + python/pyspark/sql.py | 363 ++++++++++++++++++ python/run-tests | 4 + .../org/apache/spark/sql/SQLContext.scala | 27 ++ .../org/apache/spark/sql/SchemaRDD.scala | 23 ++ .../org/apache/spark/sql/hive/TestHive.scala | 3 +- sql/hive/src/test/resources/log4j.properties | 3 + .../spark/tools/GenerateMIMAIgnore.scala | 4 +- 17 files changed, 589 insertions(+), 12 deletions(-) create mode 100644 python/pyspark/sql.py diff --git a/core/pom.xml b/core/pom.xml index a1bdd8ec68aeb..d87e2bca030e3 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -266,6 +266,11 @@ junit-interface test + + org.spark-project + pyrolite + 2.0 + target/scala-${scala.binary.version}/classes 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 32f1100406d74..f9d86fed34d0f 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 @@ -25,6 +25,8 @@ import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collectio import scala.collection.JavaConversions._ import scala.reflect.ClassTag +import net.razorvine.pickle.{Pickler, Unpickler} + import org.apache.spark._ import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import org.apache.spark.broadcast.Broadcast @@ -284,6 +286,36 @@ private[spark] object PythonRDD { file.close() } + /** + * Convert an RDD of serialized Python dictionaries to Scala Maps + * TODO: Support more Python types. + */ + def pythonToJavaMap(pyRDD: JavaRDD[Array[Byte]]): JavaRDD[Map[String, _]] = { + pyRDD.rdd.mapPartitions { iter => + val unpickle = new Unpickler + // TODO: Figure out why flatMap is necessay for pyspark + iter.flatMap { row => + unpickle.loads(row) match { + case objs: java.util.ArrayList[JMap[String, _] @unchecked] => objs.map(_.toMap) + // Incase the partition doesn't have a collection + case obj: JMap[String @unchecked, _] => Seq(obj.toMap) + } + } + } + } + + /** + * Convert and RDD of Java objects to and RDD of serialized Python objects, that is usable by + * PySpark. + */ + def javaToPython(jRDD: JavaRDD[Any]): JavaRDD[Array[Byte]] = { + jRDD.rdd.mapPartitions { iter => + val pickle = new Pickler + iter.map { row => + pickle.dumps(row) + } + } + } } private diff --git a/dev/run-tests b/dev/run-tests index 6ad674a2ba127..0725b681f1a1b 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -34,6 +34,7 @@ else fi JAVA_VERSION=$($java_cmd -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') [ "$JAVA_VERSION" -ge 18 ] && echo "" || echo "[Warn] Java 8 tests will not run because JDK version is < 1.8." +export SPARK_HIVE=true echo "=========================================================================" echo "Running Apache RAT checks" diff --git a/docs/README.md b/docs/README.md index 0678fc5c86706..75b1811ba99af 100644 --- a/docs/README.md +++ b/docs/README.md @@ -42,7 +42,7 @@ To mark a block of code in your markdown to be syntax highlighted by jekyll duri You can build just the Spark scaladoc by running `sbt/sbt doc` from the SPARK_PROJECT_ROOT directory. -Similarly, you can build just the PySpark epydoc by running `epydoc --config epydoc.conf` from the SPARK_PROJECT_ROOT/pyspark directory. +Similarly, you can build just the PySpark epydoc by running `epydoc --config epydoc.conf` from the SPARK_PROJECT_ROOT/pyspark directory. Documentation is only generated for classes that are listed as public in `__init__.py`. When you run `jekyll` in the docs directory, it will also copy over the scaladoc for the various Spark subprojects into the docs directory (and then also into the _site directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/). diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index bbd56d2fd13bb..05f0bd47a88a5 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -32,8 +32,8 @@ curr_dir = pwd cd("..") - puts "Running sbt/sbt doc from " + pwd + "; this may take a few minutes..." - puts `sbt/sbt doc` + puts "Running 'sbt/sbt doc hive/doc' from " + pwd + "; this may take a few minutes..." + puts `sbt/sbt doc hive/doc` puts "Moving back into docs dir." cd("docs") diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index a59393e1424de..6f616fb7c2448 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -20,7 +20,7 @@ a schema that describes the data types of each column in the row. A SchemaRDD i in a traditional relational database. A SchemaRDD can be created from an existing RDD, parquet file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). -**All of the examples on this page use sample data included in the Spark distribution and can be run in the spark-shell.** +**All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell`.** @@ -33,6 +33,19 @@ a schema that describes the data types of each column in the row. A JavaSchemaR in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, parquet file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). + +
      + +Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using +Spark. At the core of this component is a new type of RDD, +[SchemaRDD](api/pyspark/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed +[Row](api/pyspark/pyspark.sql.Row-class.html) objects along with +a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table +in a traditional relational database. A SchemaRDD can be created from an existing RDD, parquet +file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). + +**All of the examples on this page use sample data included in the Spark distribution and can be run in the `pyspark` shell.** +
      *************************************************************************************************** @@ -44,7 +57,7 @@ file, or by running HiveQL against data stored in [Apache Hive](http://hive.apac The entry point into all relational functionality in Spark is the [SQLContext](api/sql/core/index.html#org.apache.spark.sql.SQLContext) class, or one of its -decendents. To create a basic SQLContext, all you need is a SparkContext. +descendants. To create a basic SQLContext, all you need is a SparkContext. {% highlight scala %} val sc: SparkContext // An existing SparkContext. @@ -60,7 +73,7 @@ import sqlContext._ The entry point into all relational functionality in Spark is the [JavaSQLContext](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one -of its decendents. To create a basic JavaSQLContext, all you need is a JavaSparkContext. +of its descendants. To create a basic JavaSQLContext, all you need is a JavaSparkContext. {% highlight java %} JavaSparkContext ctx = ...; // An existing JavaSparkContext. @@ -69,6 +82,19 @@ JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(ctx); +
      + +The entry point into all relational functionality in Spark is the +[SQLContext](api/pyspark/pyspark.sql.SQLContext-class.html) class, or one +of its decedents. To create a basic SQLContext, all you need is a SparkContext. + +{% highlight python %} +from pyspark.sql import SQLContext +sqlCtx = SQLContext(sc) +{% endhighlight %} + +
      + ## Running SQL on RDDs @@ -81,7 +107,7 @@ One type of table that is supported by Spark SQL is an RDD of Scala case classes defines the schema of the table. The names of the arguments to the case class are read using reflection and become the names of the columns. Case classes can also be nested or contain complex types such as Sequences or Arrays. This RDD can be implicitly converted to a SchemaRDD and then be -registered as a table. Tables can used in subsequent SQL statements. +registered as a table. Tables can be used in subsequent SQL statements. {% highlight scala %} val sqlContext = new org.apache.spark.sql.SQLContext(sc) @@ -176,6 +202,34 @@ List teenagerNames = teenagers.map(new Function() { +
      + +One type of table that is supported by Spark SQL is an RDD of dictionaries. The keys of the +dictionary define the columns names of the table, and the types are inferred by looking at the first +row. Any RDD of dictionaries can converted to a SchemaRDD and then registered as a table. Tables +can be used in subsequent SQL statements. + +{% highlight python %} +# Load a text file and convert each line to a dictionary. +lines = sc.textFile("examples/src/main/resources/people.txt") +parts = lines.map(lambda l: l.split(",")) +people = parts.map(lambda p: {"name": p[0], "age": int(p[1])}) + +# Infer the schema, and register the SchemaRDD as a table. +# In future versions of PySpark we would like to add support for registering RDDs with other +# datatypes as tables +peopleTable = sqlCtx.inferSchema(people) +peopleTable.registerAsTable("people") + +# SQL can be run over SchemaRDDs that have been registered as a table. +teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + +# The results of SQL queries are RDDs and support all the normal RDD operations. +teenNames = teenagers.map(lambda p: "Name: " + p.name) +{% endhighlight %} + +
      + **Note that Spark SQL currently uses a very basic SQL parser.** @@ -231,6 +285,27 @@ parquetFile.registerAsTable("parquetFile"); JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); +{% endhighlight %} + + + +
      + +{% highlight python %} + +peopleTable # The SchemaRDD from the previous example. + +# SchemaRDDs can be saved as parquet files, maintaining the schema information. +peopleTable.saveAsParquetFile("people.parquet") + +# Read in the parquet file created above. Parquet files are self-describing so the schema is preserved. +# The result of loading a parquet file is also a SchemaRDD. +parquetFile = sqlCtx.parquetFile("people.parquet") + +# Parquet files can also be registered as tables and then used in SQL statements. +parquetFile.registerAsTable("parquetFile"); +teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") + {% endhighlight %}
      @@ -318,4 +393,24 @@ Row[] results = hiveCtx.hql("FROM src SELECT key, value").collect(); +
      + +When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and +adds support for finding tables in in the MetaStore and writing queries using HiveQL. In addition to +the `sql` method a `HiveContext` also provides an `hql` methods, which allows queries to be +expressed in HiveQL. + +{% highlight python %} + +from pyspark.sql import HiveContext +hiveCtx = HiveContext(sc) + +hiveCtx.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +hiveCtx.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") + +# Queries can be expressed in HiveQL. +results = hiveCtx.hql("FROM src SELECT key, value").collect() + +{% endhighlight %} +
      diff --git a/pom.xml b/pom.xml index 0eacedf7a6533..cd204376de5db 100644 --- a/pom.xml +++ b/pom.xml @@ -262,7 +262,7 @@ com.clearspring.analytics stream - 2.4.0 + 2.5.1 + + it.unimi.dsi + fastutil + + SPARK_MASTER_OPTS for master settings 4982331 [Patrick Wendell] Remove SPARK_LIBRARY_PATH 6eaf7d0 [Patrick Wendell] executorJavaOpts 0faa3b6 [Patrick Wendell] Stash of adding config options in submit script and YARN ac2d65e [Patrick Wendell] Change spark.local.dir -> SPARK_LOCAL_DIRS --- .rat-excludes | 1 + bin/run-example | 1 - bin/spark-class | 2 +- bin/spark-submit | 7 +- conf/spark-defaults.conf.template | 7 + conf/spark-env.sh.template | 43 ++-- .../scala/org/apache/spark/SparkConf.scala | 76 +++++++ .../scala/org/apache/spark/SparkContext.scala | 37 ++-- .../org/apache/spark/deploy/Client.scala | 15 +- .../org/apache/spark/deploy/Command.scala | 5 +- .../org/apache/spark/deploy/SparkSubmit.scala | 53 +++-- .../spark/deploy/SparkSubmitArguments.scala | 190 ++++++++++++++---- .../spark/deploy/client/TestClient.scala | 4 +- .../spark/deploy/worker/CommandUtils.scala | 19 +- .../spark/deploy/worker/DriverRunner.scala | 16 +- .../spark/deploy/worker/ExecutorRunner.scala | 4 +- .../org/apache/spark/executor/Executor.scala | 5 +- .../cluster/SparkDeploySchedulerBackend.scala | 13 +- .../mesos/CoarseMesosSchedulerBackend.scala | 23 ++- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- .../spark/deploy/JsonProtocolSuite.scala | 15 +- .../spark/deploy/SparkSubmitSuite.scala | 111 ++++++++-- .../deploy/worker/DriverRunnerTest.scala | 2 +- .../deploy/worker/ExecutorRunnerTest.scala | 3 +- dev/audit-release/audit_release.py | 1 + .../src/main/scala/SparkApp.scala | 7 +- .../src/main/scala/GraphxApp.scala | 10 +- .../src/main/scala/StreamingApp.scala | 8 +- docs/cluster-overview.md | 135 ++++++++----- docs/configuration.md | 50 +++-- docs/quick-start.md | 178 ++++++++-------- docs/scala-programming-guide.md | 46 +++-- docs/sql-programming-guide.md | 2 +- .../spark/deploy/yarn/ApplicationMaster.scala | 3 + .../org/apache/spark/deploy/yarn/Client.scala | 4 +- .../spark/deploy/yarn/ExecutorLauncher.scala | 6 + .../deploy/yarn/YarnAllocationHandler.scala | 4 + .../apache/spark/deploy/yarn/ClientBase.scala | 137 +++++++------ .../deploy/yarn/ExecutorRunnableUtil.scala | 21 +- .../cluster/YarnClientSchedulerBackend.scala | 4 +- .../spark/deploy/yarn/ApplicationMaster.scala | 3 + .../org/apache/spark/deploy/yarn/Client.scala | 4 +- .../spark/deploy/yarn/ExecutorLauncher.scala | 6 + .../deploy/yarn/YarnAllocationHandler.scala | 4 + 44 files changed, 886 insertions(+), 401 deletions(-) create mode 100644 conf/spark-defaults.conf.template diff --git a/.rat-excludes b/.rat-excludes index 9e9abb3f10bbf..50766954ef070 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -11,6 +11,7 @@ RELEASE control docs fairscheduler.xml.template +spark-defaults.conf.template log4j.properties log4j.properties.template metrics.properties.template diff --git a/bin/run-example b/bin/run-example index 5af95a08c6c41..b2999198a8d41 100755 --- a/bin/run-example +++ b/bin/run-example @@ -75,7 +75,6 @@ fi # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="$SPARK_JAVA_OPTS" -JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e "$FWDIR/conf/java-opts" ] ; then JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" diff --git a/bin/spark-class b/bin/spark-class index 1b0d309cc5b1c..6871e180c9fa8 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -98,7 +98,7 @@ fi # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="$OUR_JAVA_OPTS" -JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" +JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$_SPARK_LIBRARY_PATH" JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e "$FWDIR/conf/java-opts" ] ; then diff --git a/bin/spark-submit b/bin/spark-submit index d92d55a032bd5..498d0b27bacdf 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -25,8 +25,13 @@ while (($#)); do DEPLOY_MODE=$2 elif [ $1 = "--driver-memory" ]; then DRIVER_MEMORY=$2 + elif [ $1 = "--driver-library-path" ]; then + export _SPARK_LIBRARY_PATH=$2 + elif [ $1 = "--driver-class-path" ]; then + export SPARK_CLASSPATH="$SPARK_CLASSPATH:$2" + elif [ $1 = "--driver-java-options" ]; then + export SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $2" fi - shift done diff --git a/conf/spark-defaults.conf.template b/conf/spark-defaults.conf.template new file mode 100644 index 0000000000000..f840ff681d019 --- /dev/null +++ b/conf/spark-defaults.conf.template @@ -0,0 +1,7 @@ +# Default system properties included when running spark-submit. +# This is useful for setting default environmental settings. + +# Example: +# spark.master spark://master:7077 +# spark.eventLog.enabled true +# spark.eventLog.dir hdfs://namenode:8021/directory diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 6432a566089be..177a21cc0377f 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -1,22 +1,41 @@ #!/usr/bin/env bash -# This file contains environment variables required to run Spark. Copy it as -# spark-env.sh and edit that to configure Spark for your site. -# -# The following variables can be set in this file: +# This file is sourced when running various Spark programs. +# Copy it as spark-env.sh and edit that to configure Spark for your site. + +# Options read when launching programs locally with +# ./bin/run-example or ./bin/spark-submit +# - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node +# - SPARK_PUBLIC_DNS, to set the public dns name of the driver program +# - SPARK_CLASSPATH, default classpath entries to append + +# Options read by executors and drivers running inside the cluster # - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node +# - SPARK_PUBLIC_DNS, to set the public DNS name of the driver program +# - SPARK_CLASSPATH, default classpath entries to append +# - SPARK_LOCAL_DIRS, storage directories to use on this node for shuffle and RDD data # - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos -# - SPARK_JAVA_OPTS, to set node-specific JVM options for Spark. Note that -# we recommend setting app-wide options in the application's driver program. -# Examples of node-specific options : -Dspark.local.dir, GC options -# Examples of app-wide options : -Dspark.serializer -# -# If using the standalone deploy mode, you can also set variables for it here: + +# Options read in YARN client mode +# - SPARK_EXECUTOR_INSTANCES, Number of workers to start (Default: 2) +# - SPARK_EXECUTOR_CORES, Number of cores for the workers (Default: 1). +# - SPARK_EXECUTOR_MEMORY, Memory per Worker (e.g. 1000M, 2G) (Default: 1G) +# - SPARK_DRIVER_MEMORY, Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb) +# - SPARK_YARN_APP_NAME, The name of your application (Default: Spark) +# - SPARK_YARN_QUEUE, The hadoop queue to use for allocation requests (Default: ‘default’) +# - SPARK_YARN_DIST_FILES, Comma separated list of files to be distributed with the job. +# - SPARK_YARN_DIST_ARCHIVES, Comma separated list of archives to be distributed with the job. + +# Options for the daemons used in the standalone deploy mode: # - SPARK_MASTER_IP, to bind the master to a different IP address or hostname # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports +# - SPARK_MASTER_OPTS, to set config properties only for the master (e.g. "-Dx=y") # - SPARK_WORKER_CORES, to set the number of cores to use on this machine -# - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g) +# - SPARK_WORKER_MEMORY, to set how much total memory workers have to give executors (e.g. 1000m, 2g) # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT # - SPARK_WORKER_INSTANCES, to set the number of worker processes per node # - SPARK_WORKER_DIR, to set the working directory of worker processes -# - SPARK_PUBLIC_DNS, to set the public dns name of the master +# - SPARK_WORKER_OPTS, to set config properties only for the worker (e.g. "-Dx=y") +# - SPARK_HISTORY_OPTS, to set config properties only for the history server (e.g. "-Dx=y") +# - SPARK_DAEMON_OPTS, to set config properties for all daemons (e.g. "-Dx=y") +# - SPARK_PUBLIC_DNS, to set the public dns name of the master or workers diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index b947feb891ee6..bd21fdc5a18e4 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -208,6 +208,82 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { new SparkConf(false).setAll(settings) } + /** Checks for illegal or deprecated config settings. Throws an exception for the former. Not + * idempotent - may mutate this conf object to convert deprecated settings to supported ones. */ + private[spark] def validateSettings() { + if (settings.contains("spark.local.dir")) { + val msg = "In Spark 1.0 and later spark.local.dir will be overridden by the value set by " + + "the cluster manager (via SPARK_LOCAL_DIRS in mesos/standalone and LOCAL_DIRS in YARN)." + logWarning(msg) + } + + val executorOptsKey = "spark.executor.extraJavaOptions" + val executorClasspathKey = "spark.executor.extraClassPath" + val driverOptsKey = "spark.driver.extraJavaOptions" + val driverClassPathKey = "spark.driver.extraClassPath" + + // Validate spark.executor.extraJavaOptions + settings.get(executorOptsKey).map { javaOpts => + if (javaOpts.contains("-Dspark")) { + val msg = s"$executorOptsKey is not allowed to set Spark options (was '$javaOpts)'. " + + "Set them directly on a SparkConf or in a properties file when using ./bin/spark-submit." + throw new Exception(msg) + } + if (javaOpts.contains("-Xmx") || javaOpts.contains("-Xms")) { + val msg = s"$executorOptsKey is not allowed to alter memory settings (was '$javaOpts'). " + + "Use spark.executor.memory instead." + throw new Exception(msg) + } + } + + // Check for legacy configs + sys.env.get("SPARK_JAVA_OPTS").foreach { value => + val error = + s""" + |SPARK_JAVA_OPTS was detected (set to '$value'). + |This has undefined behavior when running on a cluster and is deprecated in Spark 1.0+. + | + |Please instead use: + | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application + | - ./spark-submit with --driver-java-options to set -X options for a driver + | - spark.executor.extraJavaOptions to set -X options for executors + | - SPARK_DAEMON_OPTS to set java options for standalone daemons (i.e. master, worker) + """.stripMargin + logError(error) + + for (key <- Seq(executorOptsKey, driverOptsKey)) { + if (getOption(key).isDefined) { + throw new SparkException(s"Found both $key and SPARK_JAVA_OPTS. Use only the former.") + } else { + logWarning(s"Setting '$key' to '$value' as a work-around.") + set(key, value) + } + } + } + + sys.env.get("SPARK_CLASSPATH").foreach { value => + val error = + s""" + |SPARK_CLASSPATH was detected (set to '$value'). + | This has undefined behavior when running on a cluster and is deprecated in Spark 1.0+. + | + |Please instead use: + | - ./spark-submit with --driver-class-path to augment the driver classpath + | - spark.executor.extraClassPath to augment the executor classpath + """.stripMargin + logError(error) + + for (key <- Seq(executorClasspathKey, driverClassPathKey)) { + if (getOption(key).isDefined) { + throw new SparkException(s"Found both $key and SPARK_CLASSPATH. Use only the former.") + } else { + logWarning(s"Setting '$key' to '$value' as a work-around.") + set(key, value) + } + } + } + } + /** * Return a string listing all keys and values, one per line. This is useful to print the * configuration out for debugging. diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d3ef75bc7335a..7933d68d67d96 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -148,6 +148,7 @@ class SparkContext(config: SparkConf) extends Logging { this(master, appName, sparkHome, jars, Map(), Map()) private[spark] val conf = config.clone() + conf.validateSettings() /** * Return a copy of this SparkContext's configuration. The configuration ''cannot'' be @@ -159,7 +160,7 @@ class SparkContext(config: SparkConf) extends Logging { throw new SparkException("A master URL must be set in your configuration") } if (!conf.contains("spark.app.name")) { - throw new SparkException("An application must be set in your configuration") + throw new SparkException("An application name must be set in your configuration") } if (conf.getBoolean("spark.logConf", false)) { @@ -170,11 +171,11 @@ class SparkContext(config: SparkConf) extends Logging { conf.setIfMissing("spark.driver.host", Utils.localHostName()) conf.setIfMissing("spark.driver.port", "0") - val jars: Seq[String] = if (conf.contains("spark.jars")) { - conf.get("spark.jars").split(",").filter(_.size != 0) - } else { - null - } + val jars: Seq[String] = + conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten + + val files: Seq[String] = + conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten val master = conf.get("spark.master") val appName = conf.get("spark.app.name") @@ -235,6 +236,10 @@ class SparkContext(config: SparkConf) extends Logging { jars.foreach(addJar) } + if (files != null) { + files.foreach(addFile) + } + private def warnSparkMem(value: String): String = { logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " + "deprecated, please use spark.executor.memory instead.") @@ -247,22 +252,20 @@ class SparkContext(config: SparkConf) extends Logging { .map(Utils.memoryStringToMb) .getOrElse(512) - // Environment variables to pass to our executors - private[spark] val executorEnvs = HashMap[String, String]() - for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS"); - value <- Option(System.getenv(key))) { - executorEnvs(key) = value - } + // Environment variables to pass to our executors. + // NOTE: This should only be used for test related settings. + private[spark] val testExecutorEnvs = HashMap[String, String]() + // Convert java options to env vars as a work around // since we can't set env vars directly in sbt. - for { (envKey, propKey) <- Seq(("SPARK_HOME", "spark.home"), ("SPARK_TESTING", "spark.testing")) + for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { - executorEnvs(envKey) = value + testExecutorEnvs(envKey) = value } // The Mesos scheduler backend relies on this environment variable to set executor memory. // TODO: Set this only in the Mesos scheduler. - executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" - executorEnvs ++= conf.getExecutorEnv + testExecutorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" + testExecutorEnvs ++= conf.getExecutorEnv // Set SPARK_USER for user who is running SparkContext. val sparkUser = Option { @@ -270,7 +273,7 @@ class SparkContext(config: SparkConf) extends Logging { }.getOrElse { SparkContext.SPARK_UNKNOWN_USER } - executorEnvs("SPARK_USER") = sparkUser + testExecutorEnvs("SPARK_USER") = sparkUser // Create and start the scheduler private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 8fd2c7e95b966..7ead1171525d2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -54,8 +54,21 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends System.getenv().foreach{case (k, v) => env(k) = v} val mainClass = "org.apache.spark.deploy.worker.DriverWrapper" + + val classPathConf = "spark.driver.extraClassPath" + val classPathEntries = sys.props.get(classPathConf).toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + + val libraryPathConf = "spark.driver.extraLibraryPath" + val libraryPathEntries = sys.props.get(libraryPathConf).toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + + val javaOptionsConf = "spark.driver.extraJavaOptions" + val javaOpts = sys.props.get(javaOptionsConf) val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++ - driverArgs.driverOptions, env) + driverArgs.driverOptions, env, classPathEntries, libraryPathEntries, javaOpts) val driverDescription = new DriverDescription( driverArgs.jarUrl, diff --git a/core/src/main/scala/org/apache/spark/deploy/Command.scala b/core/src/main/scala/org/apache/spark/deploy/Command.scala index fa8af9a646750..32f3ba385084f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Command.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Command.scala @@ -22,5 +22,8 @@ import scala.collection.Map private[spark] case class Command( mainClass: String, arguments: Seq[String], - environment: Map[String, String]) { + environment: Map[String, String], + classPathEntries: Seq[String], + libraryPathEntries: Seq[String], + extraJavaOptions: Option[String] = None) { } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index e5d593cade8b3..1b1e0fce0e0df 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -17,14 +17,12 @@ package org.apache.spark.deploy -import java.io.{PrintStream, File} +import java.io.{File, PrintStream} import java.net.{URI, URL} -import org.apache.spark.executor.ExecutorURLClassLoader +import scala.collection.mutable.{ArrayBuffer, HashMap, Map} -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.Map +import org.apache.spark.executor.ExecutorURLClassLoader /** * Scala code behind the spark-submit script. The script handles setting up the classpath with @@ -63,7 +61,8 @@ object SparkSubmit { /** * @return * a tuple containing the arguments for the child, a list of classpath - * entries for the child, and the main class for the child + * entries for the child, a list of system propertes, a list of env vars + * and the main class for the child */ private[spark] def createLaunchEnv(appArgs: SparkSubmitArguments): (ArrayBuffer[String], ArrayBuffer[String], Map[String, String], String) = { @@ -123,6 +122,12 @@ object SparkSubmit { val options = List[OptionAssigner]( new OptionAssigner(appArgs.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), + new OptionAssigner(appArgs.driverExtraClassPath, STANDALONE | YARN, true, + sysProp = "spark.driver.extraClassPath"), + new OptionAssigner(appArgs.driverExtraJavaOptions, STANDALONE | YARN, true, + sysProp = "spark.driver.extraJavaOptions"), + new OptionAssigner(appArgs.driverExtraLibraryPath, STANDALONE | YARN, true, + sysProp = "spark.driver.extraLibraryPath"), new OptionAssigner(appArgs.driverMemory, YARN, true, clOption = "--driver-memory"), new OptionAssigner(appArgs.name, YARN, true, clOption = "--name"), new OptionAssigner(appArgs.queue, YARN, true, clOption = "--queue"), @@ -142,10 +147,14 @@ object SparkSubmit { new OptionAssigner(appArgs.files, YARN, true, clOption = "--files"), new OptionAssigner(appArgs.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), new OptionAssigner(appArgs.archives, YARN, true, clOption = "--archives"), - new OptionAssigner(appArgs.jars, YARN, true, clOption = "--addJars") + new OptionAssigner(appArgs.jars, YARN, true, clOption = "--addJars"), + new OptionAssigner(appArgs.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), + new OptionAssigner(appArgs.jars, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.jars"), + new OptionAssigner(appArgs.name, LOCAL | STANDALONE | MESOS, false, + sysProp = "spark.app.name") ) - // more jars + // For client mode make any added jars immediately visible on the classpath if (appArgs.jars != null && !deployOnCluster) { for (jar <- appArgs.jars.split(",")) { childClasspath += jar @@ -163,6 +172,14 @@ object SparkSubmit { } } + // For standalone mode, add the application jar automatically so the user doesn't have to + // call sc.addJar. TODO: Standalone mode in the cluster + if (clusterManager == STANDALONE) { + val existingJars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq()) + sysProps.put("spark.jars", (existingJars ++ Seq(appArgs.primaryResource)).mkString(",")) + println("SPARK JARS" + sysProps.get("spark.jars")) + } + if (deployOnCluster && clusterManager == STANDALONE) { if (appArgs.supervise) { childArgs += "--supervise" @@ -173,7 +190,7 @@ object SparkSubmit { childArgs += (appArgs.master, appArgs.primaryResource, appArgs.mainClass) } - // args + // Arguments to be passed to user program if (appArgs.childArgs != null) { if (!deployOnCluster || clusterManager == STANDALONE) { childArgs ++= appArgs.childArgs @@ -184,6 +201,10 @@ object SparkSubmit { } } + for ((k, v) <- appArgs.getDefaultSparkProperties) { + if (!sysProps.contains(k)) sysProps(k) = v + } + (childArgs, childClasspath, sysProps, childMainClass) } @@ -191,11 +212,11 @@ object SparkSubmit { sysProps: Map[String, String], childMainClass: String, verbose: Boolean = false) { if (verbose) { - System.err.println(s"Main class:\n$childMainClass") - System.err.println(s"Arguments:\n${childArgs.mkString("\n")}") - System.err.println(s"System properties:\n${sysProps.mkString("\n")}") - System.err.println(s"Classpath elements:\n${childClasspath.mkString("\n")}") - System.err.println("\n") + printStream.println(s"Main class:\n$childMainClass") + printStream.println(s"Arguments:\n${childArgs.mkString("\n")}") + printStream.println(s"System properties:\n${sysProps.mkString("\n")}") + printStream.println(s"Classpath elements:\n${childClasspath.mkString("\n")}") + printStream.println("\n") } val loader = new ExecutorURLClassLoader(new Array[URL](0), @@ -226,6 +247,10 @@ object SparkSubmit { } } +/** + * Provides an indirection layer for passing arguments as system properties or flags to + * the user's driver program or to downstream launcher tools. + */ private[spark] class OptionAssigner(val value: String, val clusterManager: Int, val deployOnCluster: Boolean, diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 834b3df2f164b..02502adfbd0c4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -17,18 +17,28 @@ package org.apache.spark.deploy -import scala.collection.mutable.ArrayBuffer +import java.io.{File, FileInputStream, IOException} +import java.util.Properties + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{HashMap, ArrayBuffer} + +import org.apache.spark.SparkException /** * Parses and encapsulates arguments from the spark-submit script. */ private[spark] class SparkSubmitArguments(args: Array[String]) { - var master: String = "local" + var master: String = null var deployMode: String = null var executorMemory: String = null var executorCores: String = null var totalExecutorCores: String = null + var propertiesFile: String = null var driverMemory: String = null + var driverExtraClassPath: String = null + var driverExtraLibraryPath: String = null + var driverExtraJavaOptions: String = null var driverCores: String = null var supervise: Boolean = false var queue: String = null @@ -42,42 +52,102 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { var jars: String = null var verbose: Boolean = false - loadEnvVars() parseOpts(args.toList) + loadDefaults() + checkRequiredArguments() + + /** Return default present in the currently defined defaults file. */ + def getDefaultSparkProperties = { + val defaultProperties = new HashMap[String, String]() + if (verbose) SparkSubmit.printStream.println(s"Using properties file: $propertiesFile") + Option(propertiesFile).foreach { filename => + val file = new File(filename) + SparkSubmitArguments.getPropertiesFromFile(file).foreach { case (k, v) => + if (k.startsWith("spark")) { + defaultProperties(k) = v + if (verbose) SparkSubmit.printStream.println(s"Adding default property: $k=$v") + } + else { + SparkSubmit.printWarning(s"Ignoring non-spark config property: $k=$v") + } + } + } + defaultProperties + } + + /** Fill in any undefined values based on the current properties file or built-in defaults. */ + private def loadDefaults() = { + + // Use common defaults file, if not specified by user + if (propertiesFile == null) { + sys.env.get("SPARK_HOME").foreach { sparkHome => + val sep = File.separator + val defaultPath = s"${sparkHome}${sep}conf${sep}spark-defaults.conf" + val file = new File(defaultPath) + if (file.exists()) { + propertiesFile = file.getAbsolutePath + } + } + } + + val defaultProperties = getDefaultSparkProperties + // Use properties file as fallback for values which have a direct analog to + // arguments in this script. + master = Option(master).getOrElse(defaultProperties.get("spark.master").orNull) + executorMemory = Option(executorMemory) + .getOrElse(defaultProperties.get("spark.executor.memory").orNull) + executorCores = Option(executorCores) + .getOrElse(defaultProperties.get("spark.executor.cores").orNull) + totalExecutorCores = Option(totalExecutorCores) + .getOrElse(defaultProperties.get("spark.cores.max").orNull) + name = Option(name).getOrElse(defaultProperties.get("spark.app.name").orNull) + jars = Option(jars).getOrElse(defaultProperties.get("spark.jars").orNull) - // Sanity checks - if (args.length == 0) printUsageAndExit(-1) - if (primaryResource == null) SparkSubmit.printErrorAndExit("Must specify a primary resource") - if (mainClass == null) SparkSubmit.printErrorAndExit("Must specify a main class with --class") + // This supports env vars in older versions of Spark + master = Option(master).getOrElse(System.getenv("MASTER")) + deployMode = Option(deployMode).getOrElse(System.getenv("DEPLOY_MODE")) + + // Global defaults. These should be keep to minimum to avoid confusing behavior. + master = Option(master).getOrElse("local") + } + + /** Ensure that required fields exists. Call this only once all defaults are loaded. */ + private def checkRequiredArguments() = { + if (args.length == 0) printUsageAndExit(-1) + if (primaryResource == null) SparkSubmit.printErrorAndExit("Must specify a primary resource") + if (mainClass == null) SparkSubmit.printErrorAndExit("Must specify a main class with --class") + } override def toString = { s"""Parsed arguments: - | master $master - | deployMode $deployMode - | executorMemory $executorMemory - | executorCores $executorCores - | totalExecutorCores $totalExecutorCores - | driverMemory $driverMemory - | drivercores $driverCores - | supervise $supervise - | queue $queue - | numExecutors $numExecutors - | files $files - | archives $archives - | mainClass $mainClass - | primaryResource $primaryResource - | name $name - | childArgs [${childArgs.mkString(" ")}] - | jars $jars - | verbose $verbose + | master $master + | deployMode $deployMode + | executorMemory $executorMemory + | executorCores $executorCores + | totalExecutorCores $totalExecutorCores + | propertiesFile $propertiesFile + | driverMemory $driverMemory + | driverCores $driverCores + | driverExtraClassPath $driverExtraClassPath + | driverExtraLibraryPath $driverExtraLibraryPath + | driverExtraJavaOptions $driverExtraJavaOptions + | supervise $supervise + | queue $queue + | numExecutors $numExecutors + | files $files + | archives $archives + | mainClass $mainClass + | primaryResource $primaryResource + | name $name + | childArgs [${childArgs.mkString(" ")}] + | jars $jars + | verbose $verbose + | + |Default properties from $propertiesFile: + |${getDefaultSparkProperties.mkString(" ", "\n ", "\n")} """.stripMargin } - private def loadEnvVars() { - Option(System.getenv("MASTER")).map(master = _) - Option(System.getenv("DEPLOY_MODE")).map(deployMode = _) - } - private def parseOpts(opts: List[String]): Unit = opts match { case ("--name") :: value :: tail => name = value @@ -122,6 +192,22 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { driverCores = value parseOpts(tail) + case ("--driver-class-path") :: value :: tail => + driverExtraClassPath = value + parseOpts(tail) + + case ("--driver-java-options") :: value :: tail => + driverExtraJavaOptions = value + parseOpts(tail) + + case ("--driver-library-path") :: value :: tail => + driverExtraLibraryPath = value + parseOpts(tail) + + case ("--properties-file") :: value :: tail => + propertiesFile = value + parseOpts(tail) + case ("--supervise") :: tail => supervise = true parseOpts(tail) @@ -154,6 +240,18 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { parseOpts(tail) case value :: tail => + if (value.startsWith("-")) { + val errMessage = s"Unrecognized option '$value'." + val suggestion: Option[String] = value match { + case v if v.startsWith("--") && v.contains("=") => + val parts = v.split("=") + Some(s"Perhaps you want '${parts(0)} ${parts(1)}'?") + case _ => + None + } + SparkSubmit.printErrorAndExit(errMessage + suggestion.map(" " + _).getOrElse("")) + } + if (primaryResource != null) { val error = s"Found two conflicting resources, $value and $primaryResource." + " Expecting only one resource." @@ -178,11 +276,21 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { | --class CLASS_NAME Name of your app's main class (required for Java apps). | --arg ARG Argument to be passed to your application's main class. This | option can be specified multiple times for multiple args. - | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). | --name NAME The name of your application (Default: 'Spark'). | --jars JARS A comma-separated list of local jars to include on the | driver classpath and that SparkContext.addJar will work | with. Doesn't work on standalone with 'cluster' deploy mode. + | --files FILES Comma separated list of files to be placed in the working dir + | of each executor. + | --properties-file FILE Path to a file from which to load extra properties. If not + | specified, this will look for conf/spark-defaults.conf. + | + | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). + | --driver-java-options Extra Java options to pass to the driver + | --driver-library-path Extra library path entries to pass to the driver + | --driver-class-path Extra class path entries to pass to the driver + | + | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G). | | Spark standalone with cluster deploy mode only: | --driver-cores NUM Cores for driver (Default: 1). @@ -193,14 +301,28 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { | | YARN-only: | --executor-cores NUM Number of cores per executor (Default: 1). - | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G). | --queue QUEUE_NAME The YARN queue to submit to (Default: 'default'). | --num-executors NUM Number of executors to (Default: 2). - | --files FILES Comma separated list of files to be placed in the working dir - | of each executor. | --archives ARCHIVES Comma separated list of archives to be extracted into the | working dir of each executor.""".stripMargin ) SparkSubmit.exitFn() } } + +object SparkSubmitArguments { + /** Load properties present in the given file. */ + def getPropertiesFromFile(file: File): Seq[(String, String)] = { + require(file.exists(), s"Properties file ${file.getName} does not exist") + val inputStream = new FileInputStream(file) + val properties = new Properties() + try { + properties.load(inputStream) + } catch { + case e: IOException => + val message = s"Failed when loading Spark properties file ${file.getName}" + throw new SparkException(message, e) + } + properties.stringPropertyNames().toSeq.map(k => (k, properties(k))) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 63f166d401059..888dd45e93c6a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -49,8 +49,8 @@ private[spark] object TestClient { val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, conf = conf, securityManager = new SecurityManager(conf)) val desc = new ApplicationDescription( - "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), - Some("dummy-spark-home"), "ignored") + "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map(), Seq(), + Seq()), Some("dummy-spark-home"), "ignored") val listener = new TestListener val client = new AppClient(actorSystem, Array(url), desc, listener, new SparkConf) client.start() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 0c761dfc93a1f..9103c885fa96c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -46,21 +46,26 @@ object CommandUtils extends Logging { * the way the JAVA_OPTS are assembled there. */ def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { - val libraryOpts = getEnv("SPARK_LIBRARY_PATH", command) - .map(p => List("-Djava.library.path=" + p)) - .getOrElse(Nil) - val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")) - .map(Utils.splitCommandString).getOrElse(Nil) - val userOpts = getEnv("SPARK_JAVA_OPTS", command).map(Utils.splitCommandString).getOrElse(Nil) val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") + // Note, this will coalesce multiple options into a single command component + val extraOpts = command.extraJavaOptions.toSeq + val libraryOpts = + if (command.libraryPathEntries.size > 0) { + val joined = command.libraryPathEntries.mkString(File.pathSeparator) + Seq(s"-Djava.library.path=$joined") + } else { + Seq() + } // Figure out our classpath with the external compute-classpath script val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" val classPath = Utils.executeAndGetOutput( Seq(sparkHome + "/bin/compute-classpath" + ext), extraEnvironment=command.environment) + val userClassPath = command.classPathEntries.mkString(File.pathSeparator) + val classPathWithUser = classPath + File.pathSeparator + userClassPath - Seq("-cp", classPath) ++ libraryOpts ++ workerLocalOpts ++ userOpts ++ memoryOpts + Seq("-cp", classPathWithUser) ++ libraryOpts ++ extraOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index b4df1a0dd4718..f918b42c83bc6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.worker import java.io._ import scala.collection.JavaConversions._ -import scala.collection.mutable.Map +import scala.collection.Map import akka.actor.ActorRef import com.google.common.base.Charsets @@ -74,13 +74,17 @@ private[spark] class DriverRunner( // Make sure user application jar is on the classpath // TODO: If we add ability to submit multiple jars they should also be added here - val env = Map(driverDesc.command.environment.toSeq: _*) - env("SPARK_CLASSPATH") = env.getOrElse("SPARK_CLASSPATH", "") + s":$localJarFilename" - val newCommand = Command(driverDesc.command.mainClass, - driverDesc.command.arguments.map(substituteVariables), env) + val classPath = driverDesc.command.classPathEntries ++ Seq(s"$localJarFilename") + val newCommand = Command( + driverDesc.command.mainClass, + driverDesc.command.arguments.map(substituteVariables), + driverDesc.command.environment, + classPath, + driverDesc.command.libraryPathEntries, + driverDesc.command.extraJavaOptions) val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem, sparkHome.getAbsolutePath) - launchDriver(command, env, driverDir, driverDesc.supervise) + launchDriver(command, driverDesc.command.environment, driverDir, driverDesc.supervise) } catch { case e: Exception => finalException = Some(e) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 2edd921066876..f94cd685e8eb0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -99,7 +99,9 @@ private[spark] class ExecutorRunner( def getCommandSeq = { val command = Command(appDesc.command.mainClass, - appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment) + appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment, + appDesc.command.classPathEntries, appDesc.command.libraryPathEntries, + appDesc.command.extraJavaOptions) CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath) } 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 f89b2bffd1676..2bfb9c387e1c9 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -64,9 +64,10 @@ private[spark] class Executor( // to what Yarn on this system said was available. This will be used later when SparkEnv // created. if (java.lang.Boolean.valueOf( - System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))) - { + System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))) { conf.set("spark.local.dir", getYarnLocalDirs()) + } else if (sys.env.contains("SPARK_LOCAL_DIRS")) { + conf.set("spark.local.dir", sys.env("SPARK_LOCAL_DIRS")) } if (!isLocal) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 936e9db80573d..9544ca05dca70 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -42,11 +42,20 @@ private[spark] class SparkDeploySchedulerBackend( // The endpoint for executors to talk to us val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - conf.get("spark.driver.host"), conf.get("spark.driver.port"), + conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") + val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") + val classPathEntries = sys.props.get("spark.executor.extraClassPath").toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + val libraryPathEntries = sys.props.get("spark.executor.extraLibraryPath").toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + val command = Command( - "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) + "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.testExecutorEnvs, + classPathEntries, libraryPathEntries, extraJavaOpts) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, sparkHome, sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 06b041e1fd9a9..2cd9d6c12eaf7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -111,7 +111,18 @@ private[spark] class CoarseMesosSchedulerBackend( def createCommand(offer: Offer, numCores: Int): CommandInfo = { val environment = Environment.newBuilder() - sc.executorEnvs.foreach { case (key, value) => + val extraClassPath = conf.getOption("spark.executor.extraClassPath") + extraClassPath.foreach { cp => + environment.addVariables( + Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) + } + val extraJavaOpts = conf.getOption("spark.executor.extraJavaOptions") + + val libraryPathOption = "spark.executor.extraLibraryPath" + val extraLibraryPath = conf.getOption(libraryPathOption).map(p => s"-Djava.library.path=$p") + val extraOpts = Seq(extraJavaOpts, extraLibraryPath).flatten.mkString(" ") + + sc.testExecutorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() .setName(key) .setValue(value) @@ -123,20 +134,22 @@ private[spark] class CoarseMesosSchedulerBackend( conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) + val uri = conf.get("spark.executor.uri", null) if (uri == null) { val runScript = new File(sparkHome, "./bin/spark-class").getCanonicalPath command.setValue( - "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d".format( - runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %s %d".format( + runScript, extraOpts, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.split('/').last.split('.').head command.setValue( ("cd %s*; " + - "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d") - .format(basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %s %d") + .format(basename, extraOpts, driverUrl, offer.getSlaveId.getValue, + offer.getHostname, numCores)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } command.build() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index dfdcafe19fb93..c975f312324ed 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -90,7 +90,7 @@ private[spark] class MesosSchedulerBackend( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) val environment = Environment.newBuilder() - sc.executorEnvs.foreach { case (key, value) => + sc.testExecutorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() .setName(key) .setValue(value) diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 9f2924c23b73c..bfae32dae0dc5 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -87,7 +87,7 @@ class JsonProtocolSuite extends FunSuite { } def createAppDesc(): ApplicationDescription = { - val cmd = new Command("mainClass", List("arg1", "arg2"), Map()) + val cmd = new Command("mainClass", List("arg1", "arg2"), Map(), Seq(), Seq()) new ApplicationDescription("name", Some(4), 1234, cmd, Some("sparkHome"), "appUiUrl") } @@ -100,7 +100,7 @@ class JsonProtocolSuite extends FunSuite { def createDriverCommand() = new Command( "org.apache.spark.FakeClass", Seq("some arg --and-some options -g foo"), - Map(("K1", "V1"), ("K2", "V2")) + Map(("K1", "V1"), ("K2", "V2")), Seq("cp1", "cp2"), Seq("lp1", "lp2"), Some("-Dfoo") ) def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3, @@ -133,9 +133,12 @@ class JsonProtocolSuite extends FunSuite { def assertValidDataInJson(validateJson: JValue, expectedJson: JValue) { val Diff(c, a, d) = validateJson diff expectedJson - assert(c === JNothing, "Json changed") - assert(a === JNothing, "Json added") - assert(d === JNothing, "Json deleted") + val validatePretty = JsonMethods.pretty(validateJson) + val expectedPretty = JsonMethods.pretty(expectedJson) + val errorMessage = s"Expected:\n$expectedPretty\nFound:\n$validatePretty" + assert(c === JNothing, s"$errorMessage\nChanged:\n${JsonMethods.pretty(c)}") + assert(a === JNothing, s"$errorMessage\nAdded:\n${JsonMethods.pretty(a)}") + assert(d === JNothing, s"$errorMessage\nDelected:\n${JsonMethods.pretty(d)}") } } @@ -165,7 +168,7 @@ object JsonConstants { """ |{"name":"name","cores":4,"memoryperslave":1234, |"user":"%s","sparkhome":"sparkHome", - |"command":"Command(mainClass,List(arg1, arg2),Map())"} + |"command":"Command(mainClass,List(arg1, arg2),Map(),List(),List(),None)"} """.format(System.getProperty("user.name", "")).stripMargin val executorRunnerJsonStr = diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 4e489cd9b66a6..f82d717719b69 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -17,16 +17,16 @@ package org.apache.spark.deploy -import java.io.{OutputStream, PrintStream} +import java.io.{File, OutputStream, PrintStream} import scala.collection.mutable.ArrayBuffer +import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkException, TestUtils} +import org.apache.spark.deploy.SparkSubmit._ +import org.apache.spark.util.Utils import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import org.apache.spark.deploy.SparkSubmit._ - - class SparkSubmitSuite extends FunSuite with ShouldMatchers { val noOpOutputStream = new OutputStream { @@ -42,7 +42,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } /** Returns true if the script exits and the given search string is printed. */ - def testPrematureExit(input: Array[String], searchString: String): Boolean = { + def testPrematureExit(input: Array[String], searchString: String) = { val printStream = new BufferPrintStream() SparkSubmit.printStream = printStream @@ -60,28 +60,38 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } thread.start() thread.join() - printStream.lineBuffer.find(s => s.contains(searchString)).size > 0 + val joined = printStream.lineBuffer.mkString("\n") + if (!joined.contains(searchString)) { + fail(s"Search string '$searchString' not found in $joined") + } } test("prints usage on empty input") { - testPrematureExit(Array[String](), "Usage: spark-submit") should be (true) + testPrematureExit(Array[String](), "Usage: spark-submit") } test("prints usage with only --help") { - testPrematureExit(Array("--help"), "Usage: spark-submit") should be (true) + testPrematureExit(Array("--help"), "Usage: spark-submit") + } + + test("prints error with unrecognized option") { + testPrematureExit(Array("--blarg"), "Unrecognized option '--blarg'") + testPrematureExit(Array("-bleg"), "Unrecognized option '-bleg'") + testPrematureExit(Array("--master=abc"), + "Unrecognized option '--master=abc'. Perhaps you want '--master abc'?") } test("handles multiple binary definitions") { val adjacentJars = Array("foo.jar", "bar.jar") - testPrematureExit(adjacentJars, "error: Found two conflicting resources") should be (true) + testPrematureExit(adjacentJars, "error: Found two conflicting resources") val nonAdjacentJars = Array("foo.jar", "--master", "123", "--class", "abc", "bar.jar") - testPrematureExit(nonAdjacentJars, "error: Found two conflicting resources") should be (true) + testPrematureExit(nonAdjacentJars, "error: Found two conflicting resources") } test("handle binary specified but not class") { - testPrematureExit(Array("foo.jar"), "must specify a main class") + testPrematureExit(Array("foo.jar"), "Must specify a main class") } test("handles YARN cluster mode") { @@ -140,12 +150,11 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") - print("child args: " + childArgsStr) childArgsStr.startsWith("--memory 4g --cores 5 --supervise") should be (true) childArgsStr should include ("launch spark://h:p thejar.jar org.SomeClass arg1 arg2") mainClass should be ("org.apache.spark.deploy.Client") classpath should have length (0) - sysProps should have size (0) + sysProps should have size (1) // contains --jar entry } test("handles standalone client mode") { @@ -175,4 +184,80 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { sysProps("spark.executor.memory") should be ("5g") sysProps("spark.cores.max") should be ("5") } + + test("launch simple application with spark-submit") { + runSparkSubmit( + Seq("unUsed.jar", + "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local")) + } + + test("spark submit includes jars passed in through --jar") { + val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) + val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) + val jarsString = Seq(jar1, jar2).map(j => j.toString).mkString(",") + runSparkSubmit( + Seq("unUsed.jar", + "--class", JarCreationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local-cluster[2,1,512]", + "--jars", jarsString)) + } + + // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. + def runSparkSubmit(args: Seq[String]): String = { + val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get + Utils.executeAndGetOutput( + Seq("./bin/spark-submit") ++ args, + new File(sparkHome), + Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) + } +} + +object JarCreationTest { + def main(args: Array[String]) { + val conf = new SparkConf() + val sc = new SparkContext(conf) + val result = sc.makeRDD(1 to 100, 10).mapPartitions{ x => + var foundClasses = false + try { + Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) + Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) + foundClasses = true + } catch { + case _: Throwable => // catch all + } + Seq(foundClasses).iterator + }.collect() + if (result.contains(false)) { + throw new Exception("Could not load user defined classes inside of executors") + } + } +} + +object SimpleApplicationTest { + def main(args: Array[String]) { + val conf = new SparkConf() + val sc = new SparkContext(conf) + + val configs = Seq("spark.master", "spark.app.name") + for (config <- configs) { + val masterValue = conf.get(config) + val executorValues = sc + .makeRDD(1 to 100, 10) + .map(x => SparkEnv.get.conf.get(config)) + .collect() + .distinct + if (executorValues.size != 1) { + throw new SparkException(s"Inconsistent values for $config: $executorValues") + } + val executorValue = executorValues(0) + if (executorValue != masterValue) { + throw new SparkException( + s"Master had $config=$masterValue but executor had $config=$executorValue") + } + } + + } } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index a2c131b0c9787..4633bc3f7f25e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -29,7 +29,7 @@ import org.apache.spark.deploy.{Command, DriverDescription} class DriverRunnerTest extends FunSuite { private def createDriverRunner() = { - val command = new Command("mainClass", Seq(), Map()) + val command = new Command("mainClass", Seq(), Map(), Seq(), Seq()) val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command) new DriverRunner("driverId", new File("workDir"), new File("sparkHome"), driverDescription, null, "akka://1.2.3.4/worker/") diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 3cab8e7b37934..8ae387fa0be6f 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -27,7 +27,8 @@ class ExecutorRunnerTest extends FunSuite { test("command includes appId") { def f(s:String) = new File(s) val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")) - val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(),Map()), + val appDesc = new ApplicationDescription("app name", Some(8), 500, + Command("foo", Seq(), Map(), Seq(), Seq()), sparkHome, "appUiUrl") val appId = "12345-worker321-9876" val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome.getOrElse(".")), diff --git a/dev/audit-release/audit_release.py b/dev/audit-release/audit_release.py index fa2f02dfecc75..4a816d4101e57 100755 --- a/dev/audit-release/audit_release.py +++ b/dev/audit-release/audit_release.py @@ -114,6 +114,7 @@ def ensure_path_not_present(x): os.environ["SPARK_VERSION"] = RELEASE_VERSION os.environ["SCALA_VERSION"] = SCALA_VERSION os.environ["SPARK_RELEASE_REPOSITORY"] = RELEASE_REPOSITORY +os.environ["SPARK_AUDIT_MASTER"] = "local" for module in modules: os.environ["SPARK_MODULE"] = module ret = run_cmd("sbt clean update", exit_on_failure=False) diff --git a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala index 53fe43215e40e..a89b0d7d38bf1 100644 --- a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala +++ b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala @@ -24,8 +24,13 @@ import org.apache.spark.SparkContext._ object SimpleApp { def main(args: Array[String]) { + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple Spark App").setMaster(master) + case None => new SparkConf().setAppName("Simple Spark App") + } val logFile = "input.txt" - val sc = new SparkContext("local", "Simple App") + val sc = new SparkContext(conf) + SparkContext.jarOfClass(this.getClass).foreach(sc.addJar) val logData = sc.textFile(logFile, 2).cache() val numAs = logData.filter(line => line.contains("a")).count() val numBs = logData.filter(line => line.contains("b")).count() diff --git a/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala b/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala index da08e014ebd94..24c7f8d667296 100644 --- a/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala +++ b/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala @@ -17,14 +17,20 @@ package main.scala -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, SparkConf} import org.apache.spark.SparkContext._ import org.apache.spark.graphx._ import org.apache.spark.rdd.RDD object GraphXApp { def main(args: Array[String]) { - val sc = new SparkContext("local", "Simple GraphX App") + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple GraphX App").setMaster(master) + case None => new SparkConf().setAppName("Simple Graphx App") + } + val sc = new SparkContext(conf) + SparkContext.jarOfClass(this.getClass).foreach(sc.addJar) + val users: RDD[(VertexId, (String, String))] = sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")), (5L, ("franklin", "prof")), (2L, ("istoica", "prof")), diff --git a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala index 3d0722d2ac45e..a1d8971abe9a4 100644 --- a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala +++ b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala @@ -27,10 +27,12 @@ import org.apache.spark.streaming._ object SparkStreamingExample { def main(args: Array[String]) { - val conf = new SparkConf(true) - .setMaster("local[2]") - .setAppName("Streaming test") + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple Streaming App").setMaster(master) + case None => new SparkConf().setAppName("Simple Streaming App") + } val ssc = new StreamingContext(conf, Seconds(1)) + SparkContext.jarOfClass(this.getClass).foreach(ssc.sparkContext.addJar) val seen = ListBuffer[RDD[Int]]() val rdd1 = ssc.sparkContext.makeRDD(1 to 100, 10) diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 7f75ea44e4cea..dcc063042628c 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -50,61 +50,78 @@ The system currently supports three cluster managers: In addition, Spark's [EC2 launch scripts](ec2-scripts.html) make it easy to launch a standalone cluster on Amazon EC2. -# Launching Applications - -The recommended way to launch a compiled Spark application is through the spark-submit script (located in the -bin directory), which takes care of setting up the classpath with Spark and its dependencies, as well as -provides a layer over the different cluster managers and deploy modes that Spark supports. It's usage is - - spark-submit `` `` - -Where options are any of: - -- **\--class** - The main class to run. -- **\--master** - The URL of the cluster manager master, e.g. spark://host:port, mesos://host:port, yarn, - or local. -- **\--deploy-mode** - "client" to run the driver in the client process or "cluster" to run the driver in - a process on the cluster. For Mesos, only "client" is supported. -- **\--executor-memory** - Memory per executor (e.g. 1000M, 2G). -- **\--executor-cores** - Number of cores per executor. (Default: 2) -- **\--driver-memory** - Memory for driver (e.g. 1000M, 2G) -- **\--name** - Name of the application. -- **\--arg** - Argument to be passed to the application's main class. This option can be specified - multiple times to pass multiple arguments. -- **\--jars** - A comma-separated list of local jars to include on the driver classpath and that - SparkContext.addJar will work with. Doesn't work on standalone with 'cluster' deploy mode. - -The following currently only work for Spark standalone with cluster deploy mode: - -- **\--driver-cores** - Cores for driver (Default: 1). -- **\--supervise** - If given, restarts the driver on failure. - -The following only works for Spark standalone and Mesos only: - -- **\--total-executor-cores** - Total cores for all executors. - -The following currently only work for YARN: - -- **\--queue** - The YARN queue to place the application in. -- **\--files** - Comma separated list of files to be placed in the working dir of each executor. -- **\--archives** - Comma separated list of archives to be extracted into the working dir of each - executor. -- **\--num-executors** - Number of executors (Default: 2). - -The master and deploy mode can also be set with the MASTER and DEPLOY_MODE environment variables. -Values for these options passed via command line will override the environment variables. - -# Shipping Code to the Cluster - -The recommended way to ship your code to the cluster is to pass it through SparkContext's constructor, -which takes a list of JAR files (Java/Scala) or .egg and .zip libraries (Python) to disseminate to -worker nodes. You can also dynamically add new files to be sent to executors with `SparkContext.addJar` -and `addFile`. - -## URIs for addJar / addFile - -- **file:** - Absolute paths and `file:/` URIs are served by the driver's HTTP file server, and every executor - pulls the file from the driver HTTP server +# Bundling and Launching Applications + +### Bundling Your Application's Dependencies +If your code depends on other projects, you will need to package them alongside +your application in order to distribute the code to a Spark cluster. To do this, +to create an assembly jar (or "uber" jar) containing your code and its dependencies. Both +[sbt](https://github.com/sbt/sbt-assembly) and +[Maven](http://maven.apache.org/plugins/maven-shade-plugin/) +have assembly plugins. When creating assembly jars, list Spark and Hadoop +as `provided` dependencies; these need not be bundled since they are provided by +the cluster manager at runtime. Once you have an assembled jar you can call the `bin/spark-submit` +script as shown here while passing your jar. + +For Python, you can use the `pyFiles` argument of SparkContext +or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be distributed. + +### Launching Applications with ./bin/spark-submit + +Once a user application is bundled, it can be launched using the `spark-submit` script located in +the bin directory. This script takes care of setting up the classpath with Spark and its +dependencies, and can support different cluster managers and deploy modes that Spark supports. +It's usage is + + ./bin/spark-submit --class path.to.your.Class [other options..] + +To enumerate all options available to `spark-submit` run it with the `--help` flag. +Here are a few examples of common options: + +{% highlight bash %} +# Run application locally +./bin/spark-submit my-app.jar \ + --class my.main.ClassName + --master local[8] + +# Run on a Spark cluster +./bin/spark-submit my-app.jar \ + --class my.main.ClassName + --master spark://mycluster:7077 \ + --executor-memory 20G \ + --total-executor-cores 100 + +# Run on a YARN cluster +HADOOP_CONF_DIR=XX /bin/spark-submit my-app.jar \ + --class my.main.ClassName + --master yarn-cluster \ # can also be `yarn-client` for client mode + --executor-memory 20G \ + --num-executors 50 +{% endhighlight %} + +### Loading Configurations from a File + +The `spark-submit` script can load default `SparkConf` values from a properties file and pass them +onto your application. By default it will read configuration options from +`conf/spark-defaults.conf`. Any values specified in the file will be passed on to the +application when run. They can obviate the need for certain flags to `spark-submit`: for +instance, if `spark.master` property is set, you can safely omit the +`--master` flag from `spark-submit`. In general, configuration values explicitly set on a +`SparkConf` take the highest precedence, then flags passed to `spark-submit`, then values +in the defaults file. + +If you are ever unclear where configuration options are coming from. fine-grained debugging +information can be printed by adding the `--verbose` option to `./spark-submit`. + +### Advanced Dependency Management +When using `./bin/spark-submit` jars will be automatically transferred to the cluster. For many +users this is sufficient. However, advanced users can add jars by calling `addFile` or `addJar` +on an existing SparkContext. This can be used to distribute JAR files (Java/Scala) or .egg and +.zip libraries (Python) to executors. Spark uses the following URL scheme to allow different +strategies for disseminating jars: + +- **file:** - Absolute paths and `file:/` URIs are served by the driver's HTTP file server, and + every executor pulls the file from the driver HTTP server - **hdfs:**, **http:**, **https:**, **ftp:** - these pull down files and JARs from the URI as expected - **local:** - a URI starting with local:/ is expected to exist as a local file on each worker node. This means that no network IO will be incurred, and works well for large files/JARs that are pushed to each worker, @@ -138,6 +155,14 @@ The following table summarizes terms you'll see used to refer to cluster concept Application User program built on Spark. Consists of a driver program and executors on the cluster. + + Application jar + + A jar containing the user's Spark application. In some cases users will want to create + an "uber jar" containing their application along with its dependencies. The user's jar + should never include Hadoop or Spark libraries, however, these will be added at runtime. + + Driver program The process running the main() function of the application and creating the SparkContext diff --git a/docs/configuration.md b/docs/configuration.md index a3029837ff0cd..5a4abca2646b6 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -73,6 +73,9 @@ there are at least five properties that you will commonly want to control: Directory to use for "scratch" space in Spark, including map output files and RDDs that get stored on disk. This should be on a fast, local disk in your system. It can also be a comma-separated list of multiple directories on different disks. + + NOTE: In Spark 1.0 and later this will be overriden by SPARK_LOCAL_DIRS (Standalone, Mesos) or + LOCAL_DIRS (YARN) envrionment variables set by the cluster manager. @@ -578,7 +581,7 @@ Apart from these, the following properties are also available, and may be useful to consolidate them onto as few nodes as possible. Spreading out is usually better for data locality in HDFS, but consolidating is more efficient for compute-intensive workloads.
      Note: this setting needs to be configured in the standalone cluster master, not in individual - applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. + applications; you can set it through SPARK_MASTER_OPTS in spark-env.sh. @@ -591,7 +594,7 @@ Apart from these, the following properties are also available, and may be useful Set this lower on a shared cluster to prevent users from grabbing the whole cluster by default.
      Note: this setting needs to be configured in the standalone cluster master, not in individual - applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. + applications; you can set it through SPARK_MASTER_OPTS in spark-env.sh. @@ -649,6 +652,34 @@ Apart from these, the following properties are also available, and may be useful Number of cores to allocate for each task. + + spark.executor.extraJavaOptions + (none) + + A string of extra JVM options to pass to executors. For instance, GC settings or other + logging. Note that it is illegal to set Spark properties or heap size settings with this + option. Spark properties should be set using a SparkConf object or the + spark-defaults.conf file used with the spark-submit script. Heap size settings can be set + with spark.executor.memory. + + + + spark.executor.extraClassPath + (none) + + Extra classpath entries to append to the classpath of executors. This exists primarily + for backwards-compatibility with older versions of Spark. Users typically should not need + to set this option. + + + + spark.executor.extraLibraryPath + (none) + + Set a special library path to use when launching executor JVM's. + + + ## Viewing Spark Properties @@ -659,10 +690,9 @@ This is a useful place to check to make sure that your properties have been set # Environment Variables Certain Spark settings can be configured through environment variables, which are read from the `conf/spark-env.sh` -script in the directory where Spark is installed (or `conf/spark-env.cmd` on Windows). These variables are meant to be for machine-specific settings, such -as library search paths. While Spark properties can also be set there through `SPARK_JAVA_OPTS`, for per-application settings, we recommend setting -these properties within the application instead of in `spark-env.sh` so that different applications can use different -settings. +script in the directory where Spark is installed (or `conf/spark-env.cmd` on Windows). In Standalone and Mesos modes, +this file can give machine specific information such as hostnames. It is also sourced when running local +Spark applications or submission scripts. Note that `conf/spark-env.sh` does not exist by default when Spark is installed. However, you can copy `conf/spark-env.sh.template` to create it. Make sure you make the copy executable. @@ -672,13 +702,7 @@ The following variables can be set in `spark-env.sh`: * `JAVA_HOME`, the location where Java is installed (if it's not on your default `PATH`) * `PYSPARK_PYTHON`, the Python binary to use for PySpark * `SPARK_LOCAL_IP`, to configure which IP address of the machine to bind to. -* `SPARK_LIBRARY_PATH`, to add search directories for native libraries. -* `SPARK_CLASSPATH`, to add elements to Spark's classpath that you want to be present for _all_ applications. - Note that applications can also add dependencies for themselves through `SparkContext.addJar` -- we recommend - doing that when possible. -* `SPARK_JAVA_OPTS`, to add JVM options. This includes Java options like garbage collector settings and any system - properties that you'd like to pass with `-D`. One use case is to set some Spark properties differently on this - machine, e.g., `-Dspark.local.dir=/disk1,/disk2`. +* `SPARK_PUBLIC_DNS`, the hostname your Spark program will advertise to other machines. * Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores to use on each machine and maximum memory. diff --git a/docs/quick-start.md b/docs/quick-start.md index 60e8b1ba0eb46..6b4f4ba4254a2 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -99,23 +99,32 @@ scala> linesWithSpark.count() res9: Long = 15 {% endhighlight %} -It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark). +It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is +that these same functions can be used on very large data sets, even when they are striped across +tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to +a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark). -# A Standalone App in Scala -Now say we wanted to write a standalone application using the Spark API. We will walk through a simple application in both Scala (with SBT), Java (with Maven), and Python. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide. +# A Standalone Application +Now say we wanted to write a standalone application using the Spark API. We will walk through a +simple application in both Scala (with SBT), Java (with Maven), and Python. -We'll create a very simple Spark application in Scala. So simple, in fact, that it's named `SimpleApp.scala`: +
      +
      + +We'll create a very simple Spark application in Scala. So simple, in fact, that it's +named `SimpleApp.scala`: {% highlight scala %} /*** SimpleApp.scala ***/ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.SparkConf object SimpleApp { def main(args: Array[String]) { - val logFile = "$YOUR_SPARK_HOME/README.md" // Should be some file on your system - val sc = new SparkContext("local", "Simple App", "YOUR_SPARK_HOME", - List("target/scala-{{site.SCALA_BINARY_VERSION}}/simple-project_{{site.SCALA_BINARY_VERSION}}-1.0.jar")) + val logFile = "YOUR_SPARK_HOME/README.md" // Should be some file on your system + val conf = new SparkConf().setAppName("Simple Application") + val sc = new SparkContext(conf) val logData = sc.textFile(logFile, 2).cache() val numAs = logData.filter(line => line.contains("a")).count() val numBs = logData.filter(line => line.contains("b")).count() @@ -124,9 +133,17 @@ object SimpleApp { } {% endhighlight %} -This program just counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the program. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes. +This program just counts the number of lines containing 'a' and the number containing 'b' in the +Spark README. Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is +installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, +we initialize a SparkContext as part of the program. + +We pass the SparkContext constructor a SparkConf object which contains information about our +application. We also call sc.addJar to make sure that when our application is launched in cluster +mode, the jar file containing it will be shipped automatically to worker nodes. -This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` which explains that Spark is a dependency. This file also adds a repository that Spark depends on: +This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` +which explains that Spark is a dependency. This file also adds a repository that Spark depends on: {% highlight scala %} name := "Simple Project" @@ -140,15 +157,12 @@ libraryDependencies += "org.apache.spark" %% "spark-core" % "{{site.SPARK_VERSIO resolvers += "Akka Repository" at "http://repo.akka.io/releases/" {% endhighlight %} -If you also wish to read data from Hadoop's HDFS, you will also need to add a dependency on `hadoop-client` for your version of HDFS: - -{% highlight scala %} -libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "" -{% endhighlight %} - -Finally, for sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` according to the typical directory structure. Once that is in place, we can create a JAR package containing the application's code, then use `sbt/sbt run` to execute our program. +For sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` +according to the typical directory structure. Once that is in place, we can create a JAR package +containing the application's code, then use the `spark-submit` script to run our program. {% highlight bash %} +# Your directory layout should look like this $ find . . ./simple.sbt @@ -157,27 +171,36 @@ $ find . ./src/main/scala ./src/main/scala/SimpleApp.scala -$ sbt/sbt package -$ sbt/sbt run +# Package a jar containing your application +$ sbt package +... +[info] Packaging {..}/{..}/target/scala-2.10/simple-project_2.10-1.0.jar + +# Use spark-submit to run your application +$ YOUR_SPARK_HOME/bin/spark-submit target/scala-2.10/simple-project_2.10-1.0.jar \ + --class "SimpleApp" \ + --master local[4] ... Lines with a: 46, Lines with b: 23 {% endhighlight %} -# A Standalone App in Java -Now say we wanted to write a standalone application using the Java API. We will walk through doing this with Maven. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide. +
      +
      +This example will use Maven to compile an application jar, but any similar build system will work. We'll create a very simple Spark application, `SimpleApp.java`: {% highlight java %} /*** SimpleApp.java ***/ import org.apache.spark.api.java.*; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.Function; public class SimpleApp { public static void main(String[] args) { - String logFile = "$YOUR_SPARK_HOME/README.md"; // Should be some file on your system - JavaSparkContext sc = new JavaSparkContext("local", "Simple App", - "$YOUR_SPARK_HOME", new String[]{"target/simple-project-1.0.jar"}); + String logFile = "YOUR_SPARK_HOME/README.md"; // Should be some file on your system + SparkConf conf = new SparkConf().setAppName("Simple Application"); + JavaSparkContext sc = new JavaSparkContext(conf); JavaRDD logData = sc.textFile(logFile).cache(); long numAs = logData.filter(new Function() { @@ -193,9 +216,16 @@ public class SimpleApp { } {% endhighlight %} -This program just counts the number of lines containing 'a' and the number containing 'b' in a text file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail. +This program just counts the number of lines containing 'a' and the number containing 'b' in a text +file. Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is installed. +As with the Scala example, we initialize a SparkContext, though we use the special +`JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by +`JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes +that extend `spark.api.java.function.Function`. The +[Java programming guide](java-programming-guide.html) describes these differences in more detail. -To build the program, we also write a Maven `pom.xml` file that lists Spark as a dependency. Note that Spark artifacts are tagged with a Scala version. +To build the program, we also write a Maven `pom.xml` file that lists Spark as a dependency. +Note that Spark artifacts are tagged with a Scala version. {% highlight xml %} @@ -221,16 +251,6 @@ To build the program, we also write a Maven `pom.xml` file that lists Spark as a {% endhighlight %} -If you also wish to read data from Hadoop's HDFS, you will also need to add a dependency on `hadoop-client` for your version of HDFS: - -{% highlight xml %} - - org.apache.hadoop - hadoop-client - ... - -{% endhighlight %} - We lay out these files according to the canonical Maven directory structure: {% highlight bash %} $ find . @@ -241,16 +261,25 @@ $ find . ./src/main/java/SimpleApp.java {% endhighlight %} -Now, we can execute the application using Maven: +Now, we can package the application using Maven and execute it with `./bin/spark-submit`. {% highlight bash %} +# Package a jar containing your application $ mvn package -$ mvn exec:java -Dexec.mainClass="SimpleApp" +... +[INFO] Building jar: {..}/{..}/target/simple-project-1.0.jar + +# Use spark-submit to run your application +$ YOUR_SPARK_HOME/bin/spark-submit target/simple-project-1.0.jar \ + --class "SimpleApp" \ + --master local[4] ... Lines with a: 46, Lines with b: 23 {% endhighlight %} -# A Standalone App in Python +
      +
      + Now we will show how to write a standalone application using the Python API (PySpark). As an example, we'll create a simple Spark application, `SimpleApp.py`: @@ -259,7 +288,7 @@ As an example, we'll create a simple Spark application, `SimpleApp.py`: """SimpleApp.py""" from pyspark import SparkContext -logFile = "$YOUR_SPARK_HOME/README.md" # Should be some file on your system +logFile = "YOUR_SPARK_HOME/README.md" # Should be some file on your system sc = SparkContext("local", "Simple App") logData = sc.textFile(logFile).cache() @@ -270,11 +299,15 @@ print "Lines with a: %i, lines with b: %i" % (numAs, numBs) {% endhighlight %} -This program just counts the number of lines containing 'a' and the number containing 'b' in a text file. -Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. +This program just counts the number of lines containing 'a' and the number containing 'b' in a +text file. +Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala and Java examples, we use a SparkContext to create RDDs. -We can pass Python functions to Spark, which are automatically serialized along with any variables that they reference. -For applications that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide.html). +We can pass Python functions to Spark, which are automatically serialized along with any variables +that they reference. +For applications that use custom classes or third-party libraries, we can add those code +dependencies to SparkContext to ensure that they will be available on remote machines; this is +described in more detail in the [Python programming guide](python-programming-guide.html). `SimpleApp` is simple enough that we do not need to specify any code dependencies. We can run this application using the `bin/pyspark` script: @@ -286,57 +319,12 @@ $ ./bin/pyspark SimpleApp.py Lines with a: 46, Lines with b: 23 {% endhighlight python %} -# Running on a Cluster - -There are a few additional considerations when running applicaitons on a -[Spark](spark-standalone.html), [YARN](running-on-yarn.html), or -[Mesos](running-on-mesos.html) cluster. - -### Including Your Dependencies -If your code depends on other projects, you will need to ensure they are also -present on the slave nodes. A popular approach is to create an -assembly jar (or "uber" jar) containing your code and its dependencies. Both -[sbt](https://github.com/sbt/sbt-assembly) and -[Maven](http://maven.apache.org/plugins/maven-assembly-plugin/) -have assembly plugins. When creating assembly jars, list Spark -itself as a `provided` dependency; it need not be bundled since it is -already present on the slaves. Once you have an assembled jar, -add it to the SparkContext as shown here. It is also possible to add -your dependent jars one-by-one using the `addJar` method of `SparkContext`. - -For Python, you can use the `pyFiles` argument of SparkContext -or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be distributed. - -### Setting Configuration Options -Spark includes several [configuration options](configuration.html#spark-properties) -that influence the behavior of your application. -These should be set by building a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) -object and passing it to the SparkContext constructor. -For example, in Java and Scala, you can do: - -{% highlight scala %} -import org.apache.spark.{SparkConf, SparkContext} -val conf = new SparkConf() - .setMaster("local") - .setAppName("My application") - .set("spark.executor.memory", "1g") -val sc = new SparkContext(conf) -{% endhighlight %} - -Or in Python: - -{% highlight scala %} -from pyspark import SparkConf, SparkContext -conf = SparkConf() -conf.setMaster("local") -conf.setAppName("My application") -conf.set("spark.executor.memory", "1g")) -sc = SparkContext(conf = conf) -{% endhighlight %} +
      +
      -### Accessing Hadoop Filesystems +# Where to go from here +Congratulations on running your first Spark application! -The examples here access a local file. To read data from a distributed -filesystem, such as HDFS, include -[Hadoop version information](index.html#a-note-about-hadoop-versions) -in your build file. By default, Spark builds against HDFS 1.0.4. +* For an in-depth overview of the API see "Programming Guides" menu section. +* For running applications on a cluster head to the [deployment overview](cluster-overview.html). +* For configuration options available to Spark applications see the [configuration page](configuration.html). \ No newline at end of file diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 2b0a51e9dfc54..4431da0721ac7 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -25,36 +25,43 @@ To write a Spark application, you need to add a dependency on Spark. If you use artifactId = spark-core_{{site.SCALA_BINARY_VERSION}} version = {{site.SPARK_VERSION}} -In addition, if you wish to access an HDFS cluster, you need to add a dependency on `hadoop-client` for your version of HDFS: +In addition, if you wish to access an HDFS cluster, you need to add a dependency on +`hadoop-client` for your version of HDFS. Some common HDFS version tags are listed on the +[third party distributions](hadoop-third-party-distributions.html) page. groupId = org.apache.hadoop artifactId = hadoop-client version = -For other build systems, you can run `sbt/sbt assembly` to pack Spark and its dependencies into one JAR (`assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop*.jar`), then add this to your CLASSPATH. Set the HDFS version as described [here](index.html#a-note-about-hadoop-versions). - Finally, you need to import some Spark classes and implicit conversions into your program. Add the following lines: {% highlight scala %} import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.SparkConf {% endhighlight %} # Initializing Spark -The first thing a Spark program must do is to create a `SparkContext` object, which tells Spark how to access a cluster. -This is done through the following constructor: +The first thing a Spark program must do is to create a `SparkContext` object, which tells Spark +how to access a cluster. To create a `SparkContext` you first need to build a `SparkConf` object +that contains information about your application. {% highlight scala %} -new SparkContext(master, appName, [sparkHome], [jars]) +val conf = new SparkConf().setAppName().setMaster() +new SparkContext(conf) {% endhighlight %} -or through `new SparkContext(conf)`, which takes a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) -object for more advanced configuration. - -The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later. +The `` parameter is a string specifying a [Spark, Mesos or YARN cluster URL](#master-urls) +to connect to, or a special "local" string to run in local mode, as described below. `` is +a name for your application, which will be shown in the cluster web UI. It's also possible to set +these variables [using a configuration file](cluster-overview.html#loading-configurations-from-a-file) +which avoids hard-coding the master name in your application. -In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on exactly four cores, use +In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the +variable called `sc`. Making your own SparkContext will not work. You can set which master the +context connects to using the `MASTER` environment variable, and you can add JARs to the classpath +with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on exactly four cores, use {% highlight bash %} $ MASTER=local[4] ./bin/spark-shell @@ -83,21 +90,16 @@ The master URL passed to Spark can be in one of the following formats: The host parameter is the hostname of the Mesos master. The port must be whichever one the master is configured to use, which is 5050 by default. + yarn-client Connect to a YARN cluster in +client mode. The cluster location will be inferred based on the local Hadoop configuration. + + yarn-cluster Connect to a YARN cluster in +cluster mode. The cluster location will be inferred based on the local Hadoop configuration. + If no master URL is specified, the spark shell defaults to "local[*]". -For running on YARN, Spark launches an instance of the standalone deploy cluster within YARN; see [running on YARN](running-on-yarn.html) for details. - -### Deploying Code on a Cluster - -If you want to run your application on a cluster, you will need to specify the two optional parameters to `SparkContext` to let it find your code: - -* `sparkHome`: The path at which Spark is installed on your worker machines (it should be the same on all of them). -* `jars`: A list of JAR files on the local machine containing your application's code and any dependencies, which Spark will deploy to all the worker nodes. You'll need to package your application into a set of JARs using your build system. For example, if you're using SBT, the [sbt-assembly](https://github.com/sbt/sbt-assembly) plugin is a good way to make a single JAR with your code and dependencies. - -If you run `bin/spark-shell` on a cluster, you can add JARs to it by specifying the `ADD_JARS` environment variable before you launch it. This variable should contain a comma-separated list of JARs. For example, `ADD_JARS=a.jar,b.jar ./bin/spark-shell` will launch a shell with `a.jar` and `b.jar` on its classpath. In addition, any new classes you define in the shell will automatically be distributed. - # Resilient Distributed Datasets (RDDs) Spark revolves around the concept of a _resilient distributed dataset_ (RDD), which is a fault-tolerant collection of elements that can be operated on in parallel. There are currently two types of RDDs: *parallelized collections*, which take an existing Scala collection and run functions on it in parallel, and *Hadoop datasets*, which run functions on each record of a file in Hadoop distributed file system or any other storage system supported by Hadoop. Both types of RDDs can be operated on through the same methods. diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 6f616fb7c2448..8e98cc0c80a34 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -69,7 +69,7 @@ import sqlContext._ -
      +
      The entry point into all relational functionality in Spark is the [JavaSQLContext](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one 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 f078d06aafad0..2f74965900baf 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 @@ -39,6 +39,9 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils +/** + * An application master that runs the users driver program and allocates executors. + */ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 0179b0600c61f..00c7649e68e13 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -33,7 +33,9 @@ import org.apache.hadoop.yarn.util.{Apps, Records} import org.apache.spark.{Logging, SparkConf} - +/** + * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's alpha API. + */ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) extends YarnClientImpl with ClientBase with Logging { diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index 21f14576efe8a..ea356f33eb998 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -34,6 +34,12 @@ import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.SplitInfo +/** + * An application master that allocates executors on behalf of a driver that is running outside + * the cluster. + * + * This is used only in yarn-client mode. + */ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { 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 d6d46a5f6ce42..95f0f9d0ff2bc 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 @@ -54,6 +54,10 @@ object AllocationType extends Enumeration { // Note that right now, we assume all node asks as uniform in terms of capabilities and priority // Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for // more info on how we are requesting for containers. + +/** + * Acquires resources for executors from a ResourceManager and launches executors in new containers. + */ private[yarn] class YarnAllocationHandler( val conf: Configuration, val resourceManager: AMRMProtocol, 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 566de712fc280..c00b63669ca8e 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 @@ -22,16 +22,13 @@ import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} import java.nio.ByteBuffer import scala.collection.JavaConversions._ -import scala.collection.mutable.HashMap -import scala.collection.mutable.Map +import scala.collection.mutable.{HashMap, ListBuffer, Map} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.io.DataOutputBuffer +import org.apache.hadoop.fs.permission.FsPermission 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._ @@ -39,19 +36,18 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{Records, Apps} - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.util.Utils -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment - +import org.apache.hadoop.yarn.util.{Apps, Records} +import org.apache.spark.{Logging, SparkConf, SparkContext} /** * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The - * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster, - * which will launch a Spark master process and negotiate resources throughout its duration. + * Client submits an application to the YARN ResourceManager. + * + * Depending on the deployment mode this will launch one of two application master classes: + * 1. In standalone mode, it will launch an [[org.apache.spark.deploy.yarn.ApplicationMaster]] + * which launches a driver program inside of the cluster. + * 2. In client mode, it will launch an [[org.apache.spark.deploy.yarn.ExecutorLauncher]] to + * request executors on behalf of a driver running outside of the cluster. */ trait ClientBase extends Logging { val args: ClientArguments @@ -70,7 +66,6 @@ trait ClientBase extends Logging { // TODO(harvey): This could just go in ClientArguments. def validateArgs() = { Map( - (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!", ((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!", @@ -208,7 +203,7 @@ trait ClientBase extends Logging { val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() Map( - ClientBase.SPARK_JAR -> System.getenv("SPARK_JAR"), ClientBase.APP_JAR -> args.userJar, + ClientBase.SPARK_JAR -> ClientBase.getSparkJar, ClientBase.APP_JAR -> args.userJar, ClientBase.LOG4J_PROP -> System.getenv(ClientBase.LOG4J_CONF_ENV_KEY) ).foreach { case(destName, _localPath) => val localPath: String = if (_localPath != null) _localPath.trim() else "" @@ -251,8 +246,10 @@ trait ClientBase extends Logging { logInfo("Setting up the launch environment") val env = new HashMap[String, String]() + + val extraCp = sparkConf.getOption("spark.driver.extraClassPath") val log4jConf = System.getenv(ClientBase.LOG4J_CONF_ENV_KEY) - ClientBase.populateClasspath(args, yarnConf, sparkConf, log4jConf, env) + ClientBase.populateClasspath(yarnConf, sparkConf, log4jConf, env, extraCp) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() @@ -268,9 +265,6 @@ trait ClientBase extends Logging { YarnSparkHadoopUtil.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"), File.pathSeparator) - // Add each SPARK_* key to the environment. - System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } - env } @@ -299,13 +293,13 @@ trait ClientBase extends Logging { val amMemory = calculateAMMemory(newApp) - var JAVA_OPTS = "" + val JAVA_OPTS = ListBuffer[String]() // Add Xmx for AM memory JAVA_OPTS += "-Xmx" + amMemory + "m" val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) - JAVA_OPTS += " -Djava.io.tmpdir=" + tmpDir + JAVA_OPTS += "-Djava.io.tmpdir=" + tmpDir // TODO: Remove once cpuset version is pushed out. // The context is, default gc for server class machines ends up using all cores to do gc - @@ -319,35 +313,48 @@ trait ClientBase extends Logging { if (useConcurrentAndIncrementalGC) { // In our expts, using (default) throughput collector has severe perf ramifications in // multi-tenant machines - JAVA_OPTS += " -XX:+UseConcMarkSweepGC " - JAVA_OPTS += " -XX:+CMSIncrementalMode " - JAVA_OPTS += " -XX:+CMSIncrementalPacing " - JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 " - JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 " + JAVA_OPTS += "-XX:+UseConcMarkSweepGC" + JAVA_OPTS += "-XX:+CMSIncrementalMode" + JAVA_OPTS += "-XX:+CMSIncrementalPacing" + JAVA_OPTS += "-XX:CMSIncrementalDutyCycleMin=0" + JAVA_OPTS += "-XX:CMSIncrementalDutyCycle=10" } - if (env.isDefinedAt("SPARK_JAVA_OPTS")) { - JAVA_OPTS += " " + env("SPARK_JAVA_OPTS") + // TODO: it might be nicer to pass these as an internal environment variable rather than + // as Java options, due to complications with string parsing of nested quotes. + if (args.amClass == classOf[ExecutorLauncher].getName) { + // If we are being launched in client mode, forward the spark-conf options + // onto the executor launcher + for ((k, v) <- sparkConf.getAll) { + JAVA_OPTS += "-D" + k + "=" + "\\\"" + v + "\\\"" + } + } else { + // If we are being launched in standalone mode, capture and forward any spark + // system properties (e.g. set by spark-class). + for ((k, v) <- sys.props.filterKeys(_.startsWith("spark"))) { + JAVA_OPTS += "-D" + k + "=" + "\\\"" + v + "\\\"" + } + sys.props.get("spark.driver.extraJavaOptions").foreach(opts => JAVA_OPTS += opts) + sys.props.get("spark.driver.libraryPath").foreach(p => JAVA_OPTS += s"-Djava.library.path=$p") } JAVA_OPTS += ClientBase.getLog4jConfiguration(localResources) // Command for the ApplicationMaster - val commands = List[String]( - Environment.JAVA_HOME.$() + "/bin/java" + - " -server " + - JAVA_OPTS + - " " + args.amClass + - " --class " + args.userClass + - " --jar " + args.userJar + - userArgsToString(args) + - " --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") - - logInfo("Command for starting the Spark ApplicationMaster: " + commands(0)) - amContainer.setCommands(commands) + val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ + JAVA_OPTS ++ + Seq(args.amClass, "--class", args.userClass, "--jar ", args.userJar, + userArgsToString(args), + "--executor-memory", args.executorMemory.toString, + "--executor-cores", args.executorCores.toString, + "--num-executors ", args.numExecutors.toString, + "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", + "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") + + logInfo("Command for starting the Spark ApplicationMaster: " + commands) + + // TODO: it would be nicer to just make sure there are no null commands here + val printableCommands = commands.map(s => if (s == null) "null" else s).toList + amContainer.setCommands(printableCommands) setupSecurityToken(amContainer) amContainer @@ -361,6 +368,8 @@ object ClientBase { val LOG4J_CONF_ENV_KEY: String = "SPARK_LOG4J_CONF" val LOCAL_SCHEME = "local" + def getSparkJar = sys.env.get("SPARK_JAR").getOrElse(SparkContext.jarOfClass(this.getClass).head) + // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) { val classpathEntries = Option(conf.getStrings( @@ -433,10 +442,9 @@ object ClientBase { " -Dlog4j.configuration=" + log4jConf } - def populateClasspath(args: ClientArguments, conf: Configuration, sparkConf: SparkConf, - log4jConf: String, env: HashMap[String, String]) { - YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$(), - File.pathSeparator) + def populateClasspath(conf: Configuration, sparkConf: SparkConf, log4jConf: String, + env: HashMap[String, String], extraClassPath: Option[String] = None) { + if (log4jConf != null) { // If a custom log4j config file is provided as a local: URI, add its parent directory to the // classpath. Note that this only works if the custom config's file name is @@ -448,19 +456,26 @@ object ClientBase { File.pathSeparator) } } + + /** Add entry to the classpath. */ + def addClasspathEntry(path: String) = Apps.addToEnvironment(env, Environment.CLASSPATH.name, path) + /** Add entry to the classpath. Interpreted as a path relative to the working directory. */ + def addPwdClasspathEntry(entry: String) = addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + entry) + + extraClassPath.foreach(addClasspathEntry) + + addClasspathEntry(Environment.PWD.$()) // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = sparkConf.get("spark.yarn.user.classpath.first", "false") - .toBoolean - if (userClasspathFirst) { - addUserClasspath(args, env) - } - addClasspathEntry(System.getenv("SPARK_JAR"), SPARK_JAR, env); - ClientBase.populateHadoopClasspath(conf, env) - if (!userClasspathFirst) { - addUserClasspath(args, env) + if (sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean) { + addPwdClasspathEntry(APP_JAR) + addPwdClasspathEntry(SPARK_JAR) + ClientBase.populateHadoopClasspath(conf, env) + } else { + addPwdClasspathEntry(SPARK_JAR) + ClientBase.populateHadoopClasspath(conf, env) + addPwdClasspathEntry(APP_JAR) } - YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, - Environment.PWD.$() + Path.SEPARATOR + "*", File.pathSeparator) + addPwdClasspathEntry("*") } /** diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 40b38661f794d..7d07f6f68046a 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -19,26 +19,18 @@ package org.apache.spark.deploy.yarn import java.io.File import java.net.URI -import java.nio.ByteBuffer -import java.security.PrivilegedExceptionAction import scala.collection.JavaConversions._ import scala.collection.mutable.HashMap -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.DataOutputBuffer -import org.apache.hadoop.net.NetUtils -import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.protocolrecords._ -import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} - -import org.apache.spark.{SparkConf, Logging} import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} +import org.apache.spark.{Logging, SparkConf} trait ExecutorRunnableUtil extends Logging { @@ -58,8 +50,10 @@ trait ExecutorRunnableUtil extends Logging { // Set the JVM memory val executorMemoryString = executorMemory + "m" JAVA_OPTS += "-Xms" + executorMemoryString + " -Xmx" + executorMemoryString + " " - if (env.isDefinedAt("SPARK_JAVA_OPTS")) { - JAVA_OPTS += env("SPARK_JAVA_OPTS") + " " + + // Set extra Java options for the executor, if defined + sys.props.get("spark.executor.extraJavaOptions").foreach { opts => + JAVA_OPTS += opts } JAVA_OPTS += " -Djava.io.tmpdir=" + @@ -162,8 +156,9 @@ trait ExecutorRunnableUtil extends Logging { def prepareEnvironment: HashMap[String, String] = { val env = new HashMap[String, String]() + val extraCp = sparkConf.getOption("spark.executor.extraClassPath") val log4jConf = System.getenv(ClientBase.LOG4J_CONF_ENV_KEY) - ClientBase.populateClasspath(null, yarnConf, sparkConf, log4jConf, env) + ClientBase.populateClasspath(yarnConf, sparkConf, log4jConf, env, extraCp) if (log4jConf != null) { env(ClientBase.LOG4J_CONF_ENV_KEY) = log4jConf } 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 161918859e7c4..ce2dde0631ed9 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 @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} import org.apache.spark.{SparkException, Logging, SparkContext} -import org.apache.spark.deploy.yarn.{Client, ClientArguments} +import org.apache.spark.deploy.yarn.{Client, ClientArguments, ExecutorLauncher} import org.apache.spark.scheduler.TaskSchedulerImpl import scala.collection.mutable.ArrayBuffer @@ -54,7 +54,7 @@ private[spark] class YarnClientSchedulerBackend( "--class", "notused", "--jar", null, "--args", hostport, - "--am-class", "org.apache.spark.deploy.yarn.ExecutorLauncher" + "--am-class", classOf[ExecutorLauncher].getName ) // process any optional arguments, given either as environment variables 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 b225be6a79c0e..90e807160d4b6 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 @@ -42,6 +42,9 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils +/** + * An application master that runs the user's driver program and allocates executors. + */ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 77eb1276a0c4e..2e2fb5d4fa787 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -34,9 +34,7 @@ import org.apache.spark.{Logging, SparkConf} /** - * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The - * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster, - * which will launch a Spark master process and negotiate resources throughout its duration. + * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's stable API. */ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) extends YarnClientImpl with ClientBase with Logging { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index 67ed591c78bf9..a14bb377aa133 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -35,6 +35,12 @@ import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest +/** + * An application master that allocates executors on behalf of a driver that is running outside + * the cluster. + * + * This is used only in yarn-client mode. + */ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { 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 4fafae1aff26f..a979fe4d62630 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 @@ -56,6 +56,10 @@ object AllocationType extends Enumeration { // Note that right now, we assume all node asks as uniform in terms of capabilities and priority // Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for // more info on how we are requesting for containers. + +/** + * Acquires resources for executors from a ResourceManager and launches executors in new containers. + */ private[yarn] class YarnAllocationHandler( val conf: Configuration, val amClient: AMRMClient[ContainerRequest], From af46f1fd02b913beeac530a766f94e0c6c85674f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 21 Apr 2014 12:37:43 -0700 Subject: [PATCH 117/641] [Hot Fix] Ignore org.apache.spark.ui.UISuite tests #446 faced a connection refused exception from these tests, causing them to timeout and fail after a long time. For now, let's disable these tests. (We recently disabled the corresponding test in streaming in 7863ecca35be9af1eca0dfe5fd8806c5dd710fd6. These tests are very similar). Author: Andrew Or Closes #466 from andrewor14/ignore-ui-tests and squashes the following commits: 6f5a362 [Andrew Or] Ignore org.apache.spark.ui.UISuite tests --- core/src/test/scala/org/apache/spark/ui/UISuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index ed02b0ba00d43..fff8020ade6d1 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -36,7 +36,7 @@ import scala.xml.Node class UISuite extends FunSuite { - test("basic ui visibility") { + ignore("basic ui visibility") { withSpark(new SparkContext("local", "test")) { sc => // test if the ui is visible, and all the expected tabs are visible eventually(timeout(10 seconds), interval(50 milliseconds)) { @@ -50,7 +50,7 @@ class UISuite extends FunSuite { } } - test("visibility at localhost:4040") { + ignore("visibility at localhost:4040") { withSpark(new SparkContext("local", "test")) { sc => // test if visible from http://localhost:4040 eventually(timeout(10 seconds), interval(50 milliseconds)) { @@ -60,7 +60,7 @@ class UISuite extends FunSuite { } } - test("attaching a new tab") { + ignore("attaching a new tab") { withSpark(new SparkContext("local", "test")) { sc => val sparkUI = sc.ui From b7df31eb34523c1aaae1301b36b38a928f40e1ad Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 21 Apr 2014 12:48:02 -0700 Subject: [PATCH 118/641] SPARK-1539: RDDPage.scala contains RddPage class SPARK-1386 changed RDDPage to RddPage but didn't change the filename. I tried sbt/sbt publish-local. Inside the spark-core jar, the unit name is RDDPage.class and hence I got the following error: ~~~ [error] (run-main) java.lang.NoClassDefFoundError: org/apache/spark/ui/storage/RddPage java.lang.NoClassDefFoundError: org/apache/spark/ui/storage/RddPage at org.apache.spark.ui.SparkUI.initialize(SparkUI.scala:59) at org.apache.spark.ui.SparkUI.(SparkUI.scala:52) at org.apache.spark.ui.SparkUI.(SparkUI.scala:42) at org.apache.spark.SparkContext.(SparkContext.scala:215) at MovieLensALS$.main(MovieLensALS.scala:38) at MovieLensALS.main(MovieLensALS.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) Caused by: java.lang.ClassNotFoundException: org.apache.spark.ui.storage.RddPage at java.net.URLClassLoader$1.run(URLClassLoader.java:366) at java.net.URLClassLoader$1.run(URLClassLoader.java:355) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:354) at java.lang.ClassLoader.loadClass(ClassLoader.java:425) at java.lang.ClassLoader.loadClass(ClassLoader.java:358) at org.apache.spark.ui.SparkUI.initialize(SparkUI.scala:59) at org.apache.spark.ui.SparkUI.(SparkUI.scala:52) at org.apache.spark.ui.SparkUI.(SparkUI.scala:42) at org.apache.spark.SparkContext.(SparkContext.scala:215) at MovieLensALS$.main(MovieLensALS.scala:38) at MovieLensALS.main(MovieLensALS.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) ~~~ This can be fixed after renaming RddPage to RDDPage, or renaming RDDPage.scala to RddPage.scala. I chose the former since the name `RDD` is common in Spark code. Author: Xiangrui Meng Closes #454 from mengxr/rddpage-fix and squashes the following commits: f75e544 [Xiangrui Meng] rename RddPage to RDDPage --- core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala | 2 +- .../src/main/scala/org/apache/spark/ui/storage/StorageTab.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index d07f1c9b20fcf..2155633b8096f 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -26,7 +26,7 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ -private[ui] class RddPage(parent: StorageTab) extends WebUIPage("rdd") { +private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { private val appName = parent.appName private val basePath = parent.basePath private val listener = parent.listener diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 56429f6c07fcd..c04ef0ae37748 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -30,7 +30,7 @@ private[ui] class StorageTab(parent: SparkUI) extends WebUITab(parent, "storage" val listener = new StorageListener(parent.storageStatusListener) attachPage(new StoragePage(this)) - attachPage(new RddPage(this)) + attachPage(new RDDPage(this)) parent.registerListener(listener) } From 43e4a29dacf61bea870ed5010d6df77bc8009aa3 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Mon, 21 Apr 2014 14:10:23 -0700 Subject: [PATCH 119/641] SPARK-1399: show stage failure reason in UI https://issues.apache.org/jira/browse/SPARK-1399 refactor StageTable a bit to support additional column for failed stage Author: CodingCat Author: Nan Zhu Closes #421 from CodingCat/SPARK-1399 and squashes the following commits: 2caba36 [CodingCat] remove dummy tag 77cf305 [CodingCat] create dummy element to wrap columns 3989ce2 [CodingCat] address Aaron's comments 18fc09f [Nan Zhu] fix compile error 00ea30a [Nan Zhu] address Kay's comments 16ac83d [CodingCat] set a default value of failureReason 35df3df [CodingCat] address andrew's comments 06d21a4 [CodingCat] address andrew's comments 25a6db6 [CodingCat] style fix dc8856d [CodingCat] show stage failure reason in UI --- .../spark/ui/jobs/JobProgressPage.scala | 8 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 3 +- .../org/apache/spark/ui/jobs/StageTable.scala | 89 +++++++++++-------- 3 files changed, 61 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala index 34ff2ac34a7ca..0da62892118d4 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala @@ -41,10 +41,12 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") val now = System.currentTimeMillis val activeStagesTable = - new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent, parent.killEnabled) + new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, + parent, parent.killEnabled) val completedStagesTable = - new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) - val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) + new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent) + val failedStagesTable = + new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent) // For now, pool information is only accessible in live UIs val pools = if (live) sc.getAllPools else Seq[Schedulable]() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index fd83d37583967..0a2bf31833d2b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -40,7 +40,8 @@ private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { case Some(s) => s.values.toSeq case None => Seq[StageInfo]() } - val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) + val activeStagesTable = + new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, parent) // For now, pool information is only accessible in live UIs val pools = if (live) Seq(sc.getPoolForName(poolName).get) else Seq[Schedulable]() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 8c5b1f55fd2dc..2eb8c7a3a3073 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -27,34 +27,37 @@ import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ -private[ui] class StageTable( +private[ui] class StageTableBase( stages: Seq[StageInfo], parent: JobProgressTab, killEnabled: Boolean = false) { private val basePath = parent.basePath private val listener = parent.listener - private lazy val isFairScheduler = parent.isFairScheduler + protected def isFairScheduler = parent.isFairScheduler + + protected def columns: Seq[Node] = { + // create dummy element to wrap the columns + Stage Id ++ + {if (isFairScheduler) {Pool Name} else Seq.empty} ++ + Description + Submitted + Duration + Tasks: Succeeded/Total + Shuffle Read + Shuffle Write + } def toNodeSeq: Seq[Node] = { listener.synchronized { - stageTable(stageRow, stages) + stageTable(renderStageRow, stages) } } /** Special table that merges two header cells. */ - private def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + protected def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { - - - {if (isFairScheduler) {} else {}} - - - - - - - + {columns} {rows.map(r => makeRow(r))} @@ -94,8 +97,7 @@ private[ui] class StageTable( .getOrElse(
      {killLink}{nameLink}
      ) } - /** Render an HTML row that represents a stage */ - private def stageRow(s: StageInfo): Seq[Node] = { + protected def stageRow(s: StageInfo): Seq[Node] = { val poolName = listener.stageIdToPool.get(s.stageId) val submissionTime = s.submissionTime match { case Some(t) => UIUtils.formatDate(new Date(t)) @@ -124,25 +126,42 @@ private[ui] class StageTable( case 0 => "" case b => Utils.bytesToString(b) } - - - - {if (isFairScheduler) { - - }} - - - - ++ + {if (isFairScheduler) { + - - - + } else { + Seq.empty + }} ++ + + + + + + + } + + /** Render an HTML row that represents a stage */ + private def renderStageRow(s: StageInfo): Seq[Node] = {stageRow(s)} +} + +private[ui] class FailedStageTable( + stages: Seq[StageInfo], + parent: JobProgressTab, + killEnabled: Boolean = false) + extends StageTableBase(stages, parent, killEnabled) { + + override protected def columns: Seq[Node] = super.columns ++ + + override protected def stageRow(s: StageInfo): Seq[Node] = { + val basicColumns = super.stageRow(s) + val failureReason = + basicColumns ++ failureReason } } From 5a5b3346c79abb659260284fed0ace51942f3193 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 21 Apr 2014 14:21:17 -0700 Subject: [PATCH 120/641] Dev script: include RC name in git tag --- dev/create-release/create-release.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index bf1c5d7953bd2..fb9d9f9e072a6 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -35,7 +35,7 @@ USER_NAME=pwendell set -e -GIT_TAG=v$RELEASE_VERSION +GIT_TAG=v$RELEASE_VERSION-$RC_NAME # Artifact publishing From 04c37b6f749dc2418cc28c89964cdc687dfcbd51 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 21 Apr 2014 19:04:49 -0700 Subject: [PATCH 121/641] [SPARK-1332] Improve Spark Streaming's Network Receiver and InputDStream API [WIP] The current Network Receiver API makes it slightly complicated to right a new receiver as one needs to create an instance of BlockGenerator as shown in SocketReceiver https://github.com/apache/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala#L51 Exposing the BlockGenerator interface has made it harder to improve the receiving process. The API of NetworkReceiver (which was not a very stable API anyways) needs to be change if we are to ensure future stability. Additionally, the functions like streamingContext.socketStream that create input streams, return DStream objects. That makes it hard to expose functionality (say, rate limits) unique to input dstreams. They should return InputDStream or NetworkInputDStream. This is still not yet implemented. This PR is blocked on the graceful shutdown PR #247 Author: Tathagata Das Closes #300 from tdas/network-receiver-api and squashes the following commits: ea27b38 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into network-receiver-api 3a4777c [Tathagata Das] Renamed NetworkInputDStream to ReceiverInputDStream, and ActorReceiver related stuff. 838dd39 [Tathagata Das] Added more events to the StreamingListener to report errors and stopped receivers. a75c7a6 [Tathagata Das] Address some PR comments and fixed other issues. 91bfa72 [Tathagata Das] Fixed bugs. 8533094 [Tathagata Das] Scala style fixes. 028bde6 [Tathagata Das] Further refactored receiver to allow restarting of a receiver. 43f5290 [Tathagata Das] Made functions that create input streams return InputDStream and NetworkInputDStream, for both Scala and Java. 2c94579 [Tathagata Das] Fixed graceful shutdown by removing interrupts on receiving thread. 9e37a0b [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into network-receiver-api 3223e95 [Tathagata Das] Refactored the code that runs the NetworkReceiver into further classes and traits to make them more testable. a36cc48 [Tathagata Das] Refactored the NetworkReceiver API for future stability. --- .../scala/org/apache/spark/ui/UIUtils.scala | 6 +- .../streaming/examples/ActorWordCount.scala | 6 +- .../streaming/flume/FlumeInputDStream.scala | 28 +- .../spark/streaming/flume/FlumeUtils.scala | 10 +- .../streaming/flume/JavaFlumeStreamSuite.java | 6 +- .../streaming/flume/FlumeStreamSuite.scala | 6 +- .../streaming/kafka/KafkaInputDStream.scala | 19 +- .../spark/streaming/kafka/KafkaUtils.scala | 14 +- .../streaming/kafka/JavaKafkaStreamSuite.java | 10 +- .../streaming/kafka/KafkaStreamSuite.scala | 10 +- .../streaming/mqtt/MQTTInputDStream.scala | 41 +- .../spark/streaming/mqtt/MQTTUtils.scala | 12 +- .../streaming/mqtt/JavaMQTTStreamSuite.java | 6 +- .../streaming/mqtt/MQTTStreamSuite.scala | 6 +- .../twitter/TwitterInputDStream.scala | 23 +- .../streaming/twitter/TwitterUtils.scala | 20 +- .../twitter/TwitterStreamSuite.scala | 20 +- .../streaming/zeromq/ZeroMQReceiver.scala | 7 +- .../spark/streaming/zeromq/ZeroMQUtils.scala | 16 +- .../zeromq/JavaZeroMQStreamSuite.java | 8 +- .../streaming/zeromq/ZeroMQStreamSuite.scala | 9 +- project/MimaBuild.scala | 29 +- .../apache/spark/streaming/DStreamGraph.scala | 8 +- .../spark/streaming/StreamingContext.scala | 44 ++- .../streaming/api/java/JavaDStream.scala | 4 + .../streaming/api/java/JavaInputDStream.scala | 40 ++ .../api/java/JavaPairInputDStream.scala | 41 ++ .../java/JavaPairReceiverInputDStream.scala | 42 ++ .../api/java/JavaReceiverInputDStream.scala | 41 ++ .../api/java/JavaStreamingContext.scala | 56 ++- .../streaming/dstream/InputDStream.scala | 2 +- .../dstream/NetworkInputDStream.scala | 362 ------------------ .../dstream/PluggableInputDStream.scala | 5 +- .../streaming/dstream/RawInputDStream.scala | 16 +- .../dstream/ReceiverInputDStream.scala | 94 +++++ .../dstream/SocketInputDStream.scala | 62 ++- .../ActorReceiver.scala | 95 ++--- .../streaming/receiver/BlockGenerator.scala | 142 +++++++ .../spark/streaming/receiver/Receiver.scala | 236 ++++++++++++ .../streaming/receiver/ReceiverMessage.scala | 23 ++ .../receiver/ReceiverSupervisor.scala | 180 +++++++++ .../receiver/ReceiverSupervisorImpl.scala | 180 +++++++++ .../streaming/scheduler/JobGenerator.scala | 16 +- .../streaming/scheduler/JobScheduler.scala | 8 +- ...putTracker.scala => ReceiverTracker.scala} | 101 +++-- .../scheduler/StreamingListener.scala | 18 +- .../scheduler/StreamingListenerBus.scala | 4 + .../ui/StreamingJobProgressListener.scala | 10 +- .../spark/streaming/ui/StreamingPage.scala | 18 +- .../spark/streaming/util/RecurringTimer.scala | 4 +- .../apache/spark/streaming/JavaAPISuite.java | 9 +- .../spark/streaming/InputStreamsSuite.scala | 19 +- .../streaming/NetworkReceiverSuite.scala | 249 ++++++++++++ .../streaming/StreamingContextSuite.scala | 42 +- .../streaming/StreamingListenerSuite.scala | 84 +++- 55 files changed, 1836 insertions(+), 731 deletions(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala delete mode 100644 streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala rename streaming/src/main/scala/org/apache/spark/streaming/{receivers => receiver}/ActorReceiver.scala (66%) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala rename streaming/src/main/scala/org/apache/spark/streaming/scheduler/{NetworkInputTracker.scala => ReceiverTracker.scala} (68%) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 99770f28549c9..cf987a1ab02c3 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -121,7 +121,11 @@ private[spark] object UIUtils extends Logging { (records, "") } } - "%.1f%s".formatLocal(Locale.US, value, unit) + if (unit.isEmpty) { + "%d".formatLocal(Locale.US, value) + } else { + "%.1f%s".formatLocal(Locale.US, value, unit) + } } // Yarn has to go through a proxy so the base uri is provided and has to be on all links diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index a22e64ca3ce45..eb44768b9c101 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -26,8 +26,8 @@ import akka.actor.{Actor, ActorRef, Props, actorRef2Scala} import org.apache.spark.{SparkConf, SecurityManager} import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions -import org.apache.spark.streaming.receivers.Receiver import org.apache.spark.util.AkkaUtils +import org.apache.spark.streaming.receiver.ActorHelper case class SubscribeReceiver(receiverActor: ActorRef) case class UnsubscribeReceiver(receiverActor: ActorRef) @@ -81,14 +81,14 @@ class FeederActor extends Actor { * @see [[org.apache.spark.streaming.examples.FeederActor]] */ class SampleActorReceiver[T: ClassTag](urlOfPublisher: String) -extends Actor with Receiver { +extends Actor with ActorHelper { lazy private val remotePublisher = context.actorSelection(urlOfPublisher) override def preStart = remotePublisher ! SubscribeReceiver(context.self) def receive = { - case msg => pushBlock(msg.asInstanceOf[T]) + case msg => store(msg.asInstanceOf[T]) } override def postStop() = remotePublisher ! UnsubscribeReceiver(context.self) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index 34012b846e21e..df7605fe579f8 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -34,6 +34,8 @@ import org.apache.spark.util.Utils import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream._ +import org.apache.spark.Logging +import org.apache.spark.streaming.receiver.Receiver private[streaming] class FlumeInputDStream[T: ClassTag]( @@ -41,9 +43,9 @@ class FlumeInputDStream[T: ClassTag]( host: String, port: Int, storageLevel: StorageLevel -) extends NetworkInputDStream[SparkFlumeEvent](ssc_) { +) extends ReceiverInputDStream[SparkFlumeEvent](ssc_) { - override def getReceiver(): NetworkReceiver[SparkFlumeEvent] = { + override def getReceiver(): Receiver[SparkFlumeEvent] = { new FlumeReceiver(host, port, storageLevel) } } @@ -115,13 +117,13 @@ private[streaming] object SparkFlumeEvent { private[streaming] class FlumeEventServer(receiver : FlumeReceiver) extends AvroSourceProtocol { override def append(event : AvroFlumeEvent) : Status = { - receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event) + receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event)) Status.OK } override def appendBatch(events : java.util.List[AvroFlumeEvent]) : Status = { events.foreach (event => - receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event)) + receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event))) Status.OK } } @@ -133,23 +135,21 @@ class FlumeReceiver( host: String, port: Int, storageLevel: StorageLevel - ) extends NetworkReceiver[SparkFlumeEvent] { + ) extends Receiver[SparkFlumeEvent](storageLevel) with Logging { - lazy val blockGenerator = new BlockGenerator(storageLevel) + lazy val responder = new SpecificResponder( + classOf[AvroSourceProtocol], new FlumeEventServer(this)) + lazy val server = new NettyServer(responder, new InetSocketAddress(host, port)) - protected override def onStart() { - val responder = new SpecificResponder( - classOf[AvroSourceProtocol], new FlumeEventServer(this)) - val server = new NettyServer(responder, new InetSocketAddress(host, port)) - blockGenerator.start() + def onStart() { server.start() logInfo("Flume receiver started") } - protected override def onStop() { - blockGenerator.stop() + def onStop() { + server.close() logInfo("Flume receiver stopped") } - override def getLocationPreference = Some(host) + override def preferredLocation = Some(host) } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala index 654ba451e72fb..499f3560ef768 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -19,8 +19,8 @@ package org.apache.spark.streaming.flume import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream} -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaInputDStream, JavaStreamingContext, JavaDStream} +import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} object FlumeUtils { /** @@ -35,7 +35,7 @@ object FlumeUtils { hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[SparkFlumeEvent] = { + ): ReceiverInputDStream[SparkFlumeEvent] = { val inputStream = new FlumeInputDStream[SparkFlumeEvent](ssc, hostname, port, storageLevel) inputStream } @@ -50,7 +50,7 @@ object FlumeUtils { jssc: JavaStreamingContext, hostname: String, port: Int - ): JavaDStream[SparkFlumeEvent] = { + ): JavaReceiverInputDStream[SparkFlumeEvent] = { createStream(jssc.ssc, hostname, port) } @@ -65,7 +65,7 @@ object FlumeUtils { hostname: String, port: Int, storageLevel: StorageLevel - ): JavaDStream[SparkFlumeEvent] = { + ): JavaReceiverInputDStream[SparkFlumeEvent] = { createStream(jssc.ssc, hostname, port, storageLevel) } } diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java index 733389b98d22d..e0ad4f1015205 100644 --- a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java +++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java @@ -19,16 +19,16 @@ import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.junit.Test; public class JavaFlumeStreamSuite extends LocalJavaStreamingContext { @Test public void testFlumeStream() { // tests the API, does not actually test data receiving - JavaDStream test1 = FlumeUtils.createStream(ssc, "localhost", 12345); - JavaDStream test2 = FlumeUtils.createStream(ssc, "localhost", 12345, + JavaReceiverInputDStream test1 = FlumeUtils.createStream(ssc, "localhost", 12345); + JavaReceiverInputDStream test2 = FlumeUtils.createStream(ssc, "localhost", 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); } } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 8bc43972ab6a0..78603200d2d8a 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -31,6 +31,7 @@ import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{TestOutputStream, StreamingContext, TestSuiteBase} import org.apache.spark.streaming.util.ManualClock +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream class FlumeStreamSuite extends TestSuiteBase { @@ -39,10 +40,11 @@ class FlumeStreamSuite extends TestSuiteBase { test("flume input stream") { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) - val flumeStream = FlumeUtils.createStream(ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK) + val flumeStream: JavaReceiverInputDStream[SparkFlumeEvent] = + FlumeUtils.createStream(ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] with SynchronizedBuffer[Seq[SparkFlumeEvent]] - val outputStream = new TestOutputStream(flumeStream, outputBuffer) + val outputStream = new TestOutputStream(flumeStream.receiverInputDStream, outputBuffer) outputStream.register() ssc.start() diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index c2d9dcbfaac7a..21443ebbbfb0e 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -33,6 +33,7 @@ import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.receiver.Receiver /** * Input stream that pulls messages from a Kafka Broker. @@ -53,11 +54,11 @@ class KafkaInputDStream[ kafkaParams: Map[String, String], topics: Map[String, Int], storageLevel: StorageLevel - ) extends NetworkInputDStream[(K, V)](ssc_) with Logging { + ) extends ReceiverInputDStream[(K, V)](ssc_) with Logging { - def getReceiver(): NetworkReceiver[(K, V)] = { + def getReceiver(): Receiver[(K, V)] = { new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel) - .asInstanceOf[NetworkReceiver[(K, V)]] + .asInstanceOf[Receiver[(K, V)]] } } @@ -70,21 +71,15 @@ class KafkaReceiver[ kafkaParams: Map[String, String], topics: Map[String, Int], storageLevel: StorageLevel - ) extends NetworkReceiver[Any] { + ) extends Receiver[Any](storageLevel) with Logging { - // Handles pushing data into the BlockManager - lazy protected val blockGenerator = new BlockGenerator(storageLevel) // Connection to Kafka var consumerConnector : ConsumerConnector = null - def onStop() { - blockGenerator.stop() - } + def onStop() { } def onStart() { - blockGenerator.start() - // In case we are using multiple Threads to handle Kafka Messages val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _)) @@ -130,7 +125,7 @@ class KafkaReceiver[ def run() { logInfo("Starting MessageHandler.") for (msgAndMetadata <- stream) { - blockGenerator += (msgAndMetadata.key, msgAndMetadata.message) + store((msgAndMetadata.key, msgAndMetadata.message)) } } } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 5472d0cd04a94..86bb91f362d29 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -27,8 +27,8 @@ import kafka.serializer.{Decoder, StringDecoder} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream} -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.api.java.{JavaPairReceiverInputDStream, JavaStreamingContext, JavaPairDStream} +import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} object KafkaUtils { @@ -48,7 +48,7 @@ object KafkaUtils { groupId: String, topics: Map[String, Int], storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[(String, String)] = { + ): ReceiverInputDStream[(String, String)] = { val kafkaParams = Map[String, String]( "zookeeper.connect" -> zkQuorum, "group.id" -> groupId, "zookeeper.connection.timeout.ms" -> "10000") @@ -70,7 +70,7 @@ object KafkaUtils { kafkaParams: Map[String, String], topics: Map[String, Int], storageLevel: StorageLevel - ): DStream[(K, V)] = { + ): ReceiverInputDStream[(K, V)] = { new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel) } @@ -88,7 +88,7 @@ object KafkaUtils { zkQuorum: String, groupId: String, topics: JMap[String, JInt] - ): JavaPairDStream[String, String] = { + ): JavaPairReceiverInputDStream[String, String] = { implicit val cmt: ClassTag[String] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) @@ -110,7 +110,7 @@ object KafkaUtils { groupId: String, topics: JMap[String, JInt], storageLevel: StorageLevel - ): JavaPairDStream[String, String] = { + ): JavaPairReceiverInputDStream[String, String] = { implicit val cmt: ClassTag[String] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), @@ -139,7 +139,7 @@ object KafkaUtils { kafkaParams: JMap[String, String], topics: JMap[String, JInt], storageLevel: StorageLevel - ): JavaPairDStream[K, V] = { + ): JavaPairReceiverInputDStream[K, V] = { implicit val keyCmt: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] implicit val valueCmt: ClassTag[V] = diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 7b4999447ee69..9f8046bf00f8f 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -18,12 +18,13 @@ package org.apache.spark.streaming.kafka; import java.util.HashMap; + +import org.apache.spark.streaming.api.java.JavaPairReceiverInputDStream; import org.junit.Test; import com.google.common.collect.Maps; import kafka.serializer.StringDecoder; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaPairDStream; public class JavaKafkaStreamSuite extends LocalJavaStreamingContext { @Test @@ -31,14 +32,15 @@ public void testKafkaStream() { HashMap topics = Maps.newHashMap(); // tests the API, does not actually test data receiving - JavaPairDStream test1 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics); - JavaPairDStream test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, + JavaPairReceiverInputDStream test1 = + KafkaUtils.createStream(ssc, "localhost:12345", "group", topics); + JavaPairReceiverInputDStream test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2()); HashMap kafkaParams = Maps.newHashMap(); kafkaParams.put("zookeeper.connect", "localhost:12345"); kafkaParams.put("group.id","consumer-group"); - JavaPairDStream test3 = KafkaUtils.createStream(ssc, + JavaPairReceiverInputDStream test3 = KafkaUtils.createStream(ssc, String.class, String.class, StringDecoder.class, StringDecoder.class, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2()); } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index d9809f6409d44..e6f2c4a5cf5d1 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming.kafka import kafka.serializer.StringDecoder import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.dstream.ReceiverInputDStream class KafkaStreamSuite extends TestSuiteBase { @@ -28,10 +29,13 @@ class KafkaStreamSuite extends TestSuiteBase { val topics = Map("my-topic" -> 1) // tests the API, does not actually test data receiving - val test1 = KafkaUtils.createStream(ssc, "localhost:1234", "group", topics) - val test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2) + val test1: ReceiverInputDStream[(String, String)] = + KafkaUtils.createStream(ssc, "localhost:1234", "group", topics) + val test2: ReceiverInputDStream[(String, String)] = + KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2) val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group") - val test3 = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( + val test3: ReceiverInputDStream[(String, String)] = + KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( ssc, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2) // TODO: Actually test receiving data diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala index 1204cfba39f77..0beee8b4157ae 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala @@ -39,6 +39,7 @@ import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.receiver.Receiver /** * Input stream that subscribe messages from a Mqtt Broker. @@ -49,38 +50,36 @@ import org.apache.spark.streaming.dstream._ */ private[streaming] -class MQTTInputDStream[T: ClassTag]( +class MQTTInputDStream( @transient ssc_ : StreamingContext, brokerUrl: String, topic: String, storageLevel: StorageLevel - ) extends NetworkInputDStream[T](ssc_) with Logging { - - def getReceiver(): NetworkReceiver[T] = { - new MQTTReceiver(brokerUrl, topic, storageLevel).asInstanceOf[NetworkReceiver[T]] + ) extends ReceiverInputDStream[String](ssc_) with Logging { + + def getReceiver(): Receiver[String] = { + new MQTTReceiver(brokerUrl, topic, storageLevel) } } -private[streaming] -class MQTTReceiver(brokerUrl: String, - topic: String, - storageLevel: StorageLevel - ) extends NetworkReceiver[Any] { - lazy protected val blockGenerator = new BlockGenerator(storageLevel) +private[streaming] +class MQTTReceiver( + brokerUrl: String, + topic: String, + storageLevel: StorageLevel + ) extends Receiver[String](storageLevel) { def onStop() { - blockGenerator.stop() - } + } + def onStart() { - blockGenerator.start() - - // Set up persistence for messages - var peristance: MqttClientPersistence = new MemoryPersistence() + // Set up persistence for messages + val persistence = new MemoryPersistence() // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance - var client: MqttClient = new MqttClient(brokerUrl, MqttClient.generateClientId(), peristance) + val client = new MqttClient(brokerUrl, MqttClient.generateClientId(), persistence) // Connect to MqttBroker client.connect() @@ -89,18 +88,18 @@ class MQTTReceiver(brokerUrl: String, client.subscribe(topic) // Callback automatically triggers as and when new message arrives on specified topic - var callback: MqttCallback = new MqttCallback() { + val callback: MqttCallback = new MqttCallback() { // Handles Mqtt message override def messageArrived(arg0: String, arg1: MqttMessage) { - blockGenerator += new String(arg1.getPayload()) + store(new String(arg1.getPayload())) } override def deliveryComplete(arg0: IMqttDeliveryToken) { } override def connectionLost(arg0: Throwable) { - logInfo("Connection lost " + arg0) + restart("Connection lost ", arg0) } } diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala index 1b09ee5dc8f65..c5ffe51f9986c 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala @@ -19,9 +19,9 @@ package org.apache.spark.streaming.mqtt import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream} +import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext, JavaDStream} import scala.reflect.ClassTag -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} object MQTTUtils { /** @@ -36,8 +36,8 @@ object MQTTUtils { brokerUrl: String, topic: String, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[String] = { - new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel) + ): ReceiverInputDStream[String] = { + new MQTTInputDStream(ssc, brokerUrl, topic, storageLevel) } /** @@ -51,7 +51,7 @@ object MQTTUtils { jssc: JavaStreamingContext, brokerUrl: String, topic: String - ): JavaDStream[String] = { + ): JavaReceiverInputDStream[String] = { implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] createStream(jssc.ssc, brokerUrl, topic) } @@ -68,7 +68,7 @@ object MQTTUtils { brokerUrl: String, topic: String, storageLevel: StorageLevel - ): JavaDStream[String] = { + ): JavaReceiverInputDStream[String] = { implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] createStream(jssc.ssc, brokerUrl, topic, storageLevel) } diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java index 44743aaecf986..ce5aa1e0cdda4 100644 --- a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java +++ b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java @@ -18,7 +18,7 @@ package org.apache.spark.streaming.mqtt; import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.junit.Test; import org.apache.spark.streaming.LocalJavaStreamingContext; @@ -30,8 +30,8 @@ public void testMQTTStream() { String topic = "def"; // tests the API, does not actually test data receiving - JavaDStream test1 = MQTTUtils.createStream(ssc, brokerUrl, topic); - JavaDStream test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, + JavaReceiverInputDStream test1 = MQTTUtils.createStream(ssc, brokerUrl, topic); + JavaReceiverInputDStream test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2()); } } diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala index 89c40ad4619c9..467fd263e2d64 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.streaming.mqtt import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.dstream.ReceiverInputDStream class MQTTStreamSuite extends TestSuiteBase { @@ -28,8 +29,9 @@ class MQTTStreamSuite extends TestSuiteBase { val topic = "def" // tests the API, does not actually test data receiving - val test1 = MQTTUtils.createStream(ssc, brokerUrl, topic) - val test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2) + val test1: ReceiverInputDStream[String] = MQTTUtils.createStream(ssc, brokerUrl, topic) + val test2: ReceiverInputDStream[String] = + MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2) // TODO: Actually test receiving data ssc.stop() diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala index 843a4a7a9ad72..7bca1407116fa 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala @@ -25,6 +25,8 @@ import twitter4j.auth.OAuthAuthorization import org.apache.spark.streaming._ import org.apache.spark.streaming.dstream._ import org.apache.spark.storage.StorageLevel +import org.apache.spark.Logging +import org.apache.spark.streaming.receiver.Receiver /* A stream of Twitter statuses, potentially filtered by one or more keywords. * @@ -41,7 +43,7 @@ class TwitterInputDStream( twitterAuth: Option[Authorization], filters: Seq[String], storageLevel: StorageLevel - ) extends NetworkInputDStream[Status](ssc_) { + ) extends ReceiverInputDStream[Status](ssc_) { private def createOAuthAuthorization(): Authorization = { new OAuthAuthorization(new ConfigurationBuilder().build()) @@ -49,7 +51,7 @@ class TwitterInputDStream( private val authorization = twitterAuth.getOrElse(createOAuthAuthorization()) - override def getReceiver(): NetworkReceiver[Status] = { + override def getReceiver(): Receiver[Status] = { new TwitterReceiver(authorization, filters, storageLevel) } } @@ -59,27 +61,27 @@ class TwitterReceiver( twitterAuth: Authorization, filters: Seq[String], storageLevel: StorageLevel - ) extends NetworkReceiver[Status] { + ) extends Receiver[Status](storageLevel) with Logging { var twitterStream: TwitterStream = _ - lazy val blockGenerator = new BlockGenerator(storageLevel) - protected override def onStart() { - blockGenerator.start() + def onStart() { twitterStream = new TwitterStreamFactory().getInstance(twitterAuth) twitterStream.addListener(new StatusListener { def onStatus(status: Status) = { - blockGenerator += status + store(status) } // Unimplemented def onDeletionNotice(statusDeletionNotice: StatusDeletionNotice) {} def onTrackLimitationNotice(i: Int) {} def onScrubGeo(l: Long, l1: Long) {} def onStallWarning(stallWarning: StallWarning) {} - def onException(e: Exception) { stopOnError(e) } + def onException(e: Exception) { + restart("Error receiving tweets", e) + } }) - val query: FilterQuery = new FilterQuery + val query = new FilterQuery if (filters.size > 0) { query.track(filters.toArray) twitterStream.filter(query) @@ -89,8 +91,7 @@ class TwitterReceiver( logInfo("Twitter receiver started") } - protected override def onStop() { - blockGenerator.stop() + def onStop() { twitterStream.shutdown() logInfo("Twitter receiver stopped") } diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala index e8433b7e9f6fa..c6a9a2b73714f 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala @@ -21,8 +21,8 @@ import twitter4j.Status import twitter4j.auth.Authorization import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaDStream, JavaStreamingContext} +import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} object TwitterUtils { /** @@ -40,7 +40,7 @@ object TwitterUtils { twitterAuth: Option[Authorization], filters: Seq[String] = Nil, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[Status] = { + ): ReceiverInputDStream[Status] = { new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel) } @@ -52,7 +52,7 @@ object TwitterUtils { * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. * @param jssc JavaStreamingContext object */ - def createStream(jssc: JavaStreamingContext): JavaDStream[Status] = { + def createStream(jssc: JavaStreamingContext): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, None) } @@ -65,7 +65,8 @@ object TwitterUtils { * @param jssc JavaStreamingContext object * @param filters Set of filter strings to get only those tweets that match them */ - def createStream(jssc: JavaStreamingContext, filters: Array[String]): JavaDStream[Status] = { + def createStream(jssc: JavaStreamingContext, filters: Array[String] + ): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, None, filters) } @@ -82,7 +83,7 @@ object TwitterUtils { jssc: JavaStreamingContext, filters: Array[String], storageLevel: StorageLevel - ): JavaDStream[Status] = { + ): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, None, filters, storageLevel) } @@ -92,7 +93,8 @@ object TwitterUtils { * @param jssc JavaStreamingContext object * @param twitterAuth Twitter4J Authorization */ - def createStream(jssc: JavaStreamingContext, twitterAuth: Authorization): JavaDStream[Status] = { + def createStream(jssc: JavaStreamingContext, twitterAuth: Authorization + ): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, Some(twitterAuth)) } @@ -107,7 +109,7 @@ object TwitterUtils { jssc: JavaStreamingContext, twitterAuth: Authorization, filters: Array[String] - ): JavaDStream[Status] = { + ): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, Some(twitterAuth), filters) } @@ -123,7 +125,7 @@ object TwitterUtils { twitterAuth: Authorization, filters: Array[String], storageLevel: StorageLevel - ): JavaDStream[Status] = { + ): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, Some(twitterAuth), filters, storageLevel) } } diff --git a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala index 06ab0cdaf3b4e..93741e0375164 100644 --- a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala +++ b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.streaming.twitter import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} import org.apache.spark.storage.StorageLevel import twitter4j.auth.{NullAuthorization, Authorization} +import org.apache.spark.streaming.dstream.ReceiverInputDStream +import twitter4j.Status class TwitterStreamSuite extends TestSuiteBase { @@ -29,13 +31,17 @@ class TwitterStreamSuite extends TestSuiteBase { val authorization: Authorization = NullAuthorization.getInstance() // tests the API, does not actually test data receiving - val test1 = TwitterUtils.createStream(ssc, None) - val test2 = TwitterUtils.createStream(ssc, None, filters) - val test3 = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_AND_DISK_SER_2) - val test4 = TwitterUtils.createStream(ssc, Some(authorization)) - val test5 = TwitterUtils.createStream(ssc, Some(authorization), filters) - val test6 = TwitterUtils.createStream(ssc, Some(authorization), filters, - StorageLevel.MEMORY_AND_DISK_SER_2) + val test1: ReceiverInputDStream[Status] = TwitterUtils.createStream(ssc, None) + val test2: ReceiverInputDStream[Status] = + TwitterUtils.createStream(ssc, None, filters) + val test3: ReceiverInputDStream[Status] = + TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_AND_DISK_SER_2) + val test4: ReceiverInputDStream[Status] = + TwitterUtils.createStream(ssc, Some(authorization)) + val test5: ReceiverInputDStream[Status] = + TwitterUtils.createStream(ssc, Some(authorization), filters) + val test6: ReceiverInputDStream[Status] = TwitterUtils.createStream( + ssc, Some(authorization), filters, StorageLevel.MEMORY_AND_DISK_SER_2) // Note that actually testing the data receiving is hard as authentication keys are // necessary for accessing Twitter live stream diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala index a538c38dc4d6f..554705878ee78 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala @@ -24,7 +24,7 @@ import akka.util.ByteString import akka.zeromq._ import org.apache.spark.Logging -import org.apache.spark.streaming.receivers._ +import org.apache.spark.streaming.receiver.ActorHelper /** * A receiver to subscribe to ZeroMQ stream. @@ -32,7 +32,7 @@ import org.apache.spark.streaming.receivers._ private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, subscribe: Subscribe, bytesToObjects: Seq[ByteString] => Iterator[T]) - extends Actor with Receiver with Logging { + extends Actor with ActorHelper with Logging { override def preStart() = ZeroMQExtension(context.system) .newSocket(SocketType.Sub, Listener(self), Connect(publisherUrl), subscribe) @@ -46,9 +46,8 @@ private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, // We ignore first frame for processing as it is the topic val bytes = m.frames.tail - pushBlock(bytesToObjects(bytes)) + store(bytesToObjects(bytes)) case Closed => logInfo("received closed ") - } } diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala index b254e00714621..0469d0af8864a 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala @@ -24,10 +24,10 @@ import akka.util.ByteString import akka.zeromq.Subscribe import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.receivers.ReceiverSupervisorStrategy import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream} -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} +import org.apache.spark.streaming.dstream.{ReceiverInputDStream} +import org.apache.spark.streaming.receiver.ActorSupervisorStrategy object ZeroMQUtils { /** @@ -48,8 +48,8 @@ object ZeroMQUtils { subscribe: Subscribe, bytesToObjects: Seq[ByteString] => Iterator[T], storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, - supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy - ): DStream[T] = { + supervisorStrategy: SupervisorStrategy = ActorSupervisorStrategy.defaultStrategy + ): ReceiverInputDStream[T] = { ssc.actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)), "ZeroMQReceiver", storageLevel, supervisorStrategy) } @@ -72,7 +72,7 @@ object ZeroMQUtils { bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], storageLevel: StorageLevel, supervisorStrategy: SupervisorStrategy - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val fn = (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).toIterator @@ -96,7 +96,7 @@ object ZeroMQUtils { subscribe: Subscribe, bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], storageLevel: StorageLevel - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val fn = (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).toIterator @@ -119,7 +119,7 @@ object ZeroMQUtils { publisherUrl: String, subscribe: Subscribe, bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]] - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val fn = (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).toIterator diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java index d2361e14b898a..417b91eecb0ee 100644 --- a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java +++ b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming.zeromq; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.junit.Test; import akka.actor.SupervisorStrategy; import akka.util.ByteString; @@ -24,7 +25,6 @@ import org.apache.spark.api.java.function.Function; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaDStream; public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext { @@ -39,11 +39,11 @@ public Iterable call(byte[][] bytes) throws Exception { } }; - JavaDStream test1 = ZeroMQUtils.createStream( + JavaReceiverInputDStream test1 = ZeroMQUtils.createStream( ssc, publishUrl, subscribe, bytesToObjects); - JavaDStream test2 = ZeroMQUtils.createStream( + JavaReceiverInputDStream test2 = ZeroMQUtils.createStream( ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaDStream test3 = ZeroMQUtils.createStream( + JavaReceiverInputDStream test3 = ZeroMQUtils.createStream( ssc,publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(), SupervisorStrategy.defaultStrategy()); } diff --git a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala index 92d55a7a7b6e4..cc10ff6ae03cd 100644 --- a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala +++ b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala @@ -23,6 +23,7 @@ import akka.zeromq.Subscribe import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} +import org.apache.spark.streaming.dstream.ReceiverInputDStream class ZeroMQStreamSuite extends TestSuiteBase { @@ -33,10 +34,12 @@ class ZeroMQStreamSuite extends TestSuiteBase { val bytesToObjects = (bytes: Seq[ByteString]) => null.asInstanceOf[Iterator[String]] // tests the API, does not actually test data receiving - val test1 = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects) - val test2 = ZeroMQUtils.createStream( + val test1: ReceiverInputDStream[String] = + ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects) + val test2: ReceiverInputDStream[String] = ZeroMQUtils.createStream( ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2) - val test3 = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects, + val test3: ReceiverInputDStream[String] = ZeroMQUtils.createStream( + ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2, SupervisorStrategy.defaultStrategy) // TODO: Actually test data receiving diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 9cb31d70444ff..d540dc0a986e9 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -38,6 +38,7 @@ object MimaBuild { IO.read(excludeFile).split("\n") } + // Exclude a single class and its corresponding object def excludeClass(className: String) = { Seq( excludePackage(className), @@ -48,7 +49,16 @@ object MimaBuild { ProblemFilters.exclude[MissingTypesProblem](className + "$") ) } - def excludeSparkClass(className: String) = excludeClass("org.apache.spark." + className) + + // Exclude a Spark class, that is in the package org.apache.spark + def excludeSparkClass(className: String) = { + excludeClass("org.apache.spark." + className) + } + + // Exclude a Spark package, that is in the package org.apache.spark + def excludeSparkPackage(packageName: String) = { + excludePackage("org.apache.spark." + packageName) + } val packagePrivateExcludes = packagePrivateList.flatMap(excludeClass) @@ -58,10 +68,9 @@ object MimaBuild { SparkBuild.SPARK_VERSION match { case v if v.startsWith("1.0") => Seq( - excludePackage("org.apache.spark.api.java"), - excludePackage("org.apache.spark.streaming.api.java"), - excludePackage("org.apache.spark.streaming.scheduler"), - excludePackage("org.apache.spark.mllib") + excludeSparkPackage("api.java"), + excludeSparkPackage("mllib"), + excludeSparkPackage("streaming") ) ++ excludeSparkClass("rdd.ClassTags") ++ excludeSparkClass("util.XORShiftRandom") ++ @@ -69,14 +78,7 @@ object MimaBuild { excludeSparkClass("mllib.optimization.SquaredGradient") ++ excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ excludeSparkClass("mllib.regression.LassoWithSGD") ++ - excludeSparkClass("mllib.regression.LinearRegressionWithSGD") ++ - excludeSparkClass("streaming.dstream.NetworkReceiver") ++ - excludeSparkClass("streaming.dstream.NetworkReceiver#NetworkReceiverActor") ++ - excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator") ++ - excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator#Block") ++ - excludeSparkClass("streaming.dstream.ReportError") ++ - excludeSparkClass("streaming.dstream.ReportBlock") ++ - excludeSparkClass("streaming.dstream.DStream") + excludeSparkClass("mllib.regression.LinearRegressionWithSGD") case _ => Seq() } @@ -87,5 +89,4 @@ object MimaBuild { previousArtifact := None, binaryIssueFilters ++= ignoredABIProblems(sparkHome) ) - } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index d3339063cc079..b4adf0e9651a8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer import java.io.{ObjectInputStream, IOException, ObjectOutputStream} import org.apache.spark.Logging import org.apache.spark.streaming.scheduler.Job -import org.apache.spark.streaming.dstream.{DStream, NetworkInputDStream, InputDStream} +import org.apache.spark.streaming.dstream.{DStream, ReceiverInputDStream, InputDStream} final private[streaming] class DStreamGraph extends Serializable with Logging { @@ -103,9 +103,9 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { def getOutputStreams() = this.synchronized { outputStreams.toArray } - def getNetworkInputStreams() = this.synchronized { - inputStreams.filter(_.isInstanceOf[NetworkInputDStream[_]]) - .map(_.asInstanceOf[NetworkInputDStream[_]]) + def getReceiverInputStreams() = this.synchronized { + inputStreams.filter(_.isInstanceOf[ReceiverInputDStream[_]]) + .map(_.asInstanceOf[ReceiverInputDStream[_]]) .toArray } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index e9a4f7ba22576..daa5c69bbadbf 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -31,12 +31,11 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat - import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.receivers._ +import org.apache.spark.streaming.receiver.{ActorSupervisorStrategy, ActorReceiver, Receiver} import org.apache.spark.streaming.scheduler._ import org.apache.spark.streaming.ui.StreamingTab import org.apache.spark.util.MetadataCleaner @@ -139,7 +138,7 @@ class StreamingContext private[streaming] ( } } - private val nextNetworkInputStreamId = new AtomicInteger(0) + private val nextReceiverInputStreamId = new AtomicInteger(0) private[streaming] var checkpointDir: String = { if (isCheckpointPresent) { @@ -208,15 +207,26 @@ class StreamingContext private[streaming] ( if (isCheckpointPresent) cp_ else null } - private[streaming] def getNewNetworkStreamId() = nextNetworkInputStreamId.getAndIncrement() + private[streaming] def getNewReceiverStreamId() = nextReceiverInputStreamId.getAndIncrement() /** - * Create an input stream with any arbitrary user implemented network receiver. + * Create an input stream with any arbitrary user implemented receiver. * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html - * @param receiver Custom implementation of NetworkReceiver + * @param receiver Custom implementation of Receiver */ + @deprecated("Use receiverStream", "1.0.0") def networkStream[T: ClassTag]( - receiver: NetworkReceiver[T]): DStream[T] = { + receiver: Receiver[T]): ReceiverInputDStream[T] = { + receiverStream(receiver) + } + + /** + * Create an input stream with any arbitrary user implemented receiver. + * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html + * @param receiver Custom implementation of Receiver + */ + def receiverStream[T: ClassTag]( + receiver: Receiver[T]): ReceiverInputDStream[T] = { new PluggableInputDStream[T](this, receiver) } @@ -236,9 +246,9 @@ class StreamingContext private[streaming] ( props: Props, name: String, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, - supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy - ): DStream[T] = { - networkStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy)) + supervisorStrategy: SupervisorStrategy = ActorSupervisorStrategy.defaultStrategy + ): ReceiverInputDStream[T] = { + receiverStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy)) } /** @@ -254,7 +264,7 @@ class StreamingContext private[streaming] ( hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[String] = { + ): ReceiverInputDStream[String] = { socketStream[String](hostname, port, SocketReceiver.bytesToLines, storageLevel) } @@ -273,7 +283,7 @@ class StreamingContext private[streaming] ( port: Int, converter: (InputStream) => Iterator[T], storageLevel: StorageLevel - ): DStream[T] = { + ): ReceiverInputDStream[T] = { new SocketInputDStream[T](this, hostname, port, converter, storageLevel) } @@ -292,7 +302,7 @@ class StreamingContext private[streaming] ( hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[T] = { + ): ReceiverInputDStream[T] = { new RawInputDStream[T](this, hostname, port, storageLevel) } @@ -310,7 +320,7 @@ class StreamingContext private[streaming] ( K: ClassTag, V: ClassTag, F <: NewInputFormat[K, V]: ClassTag - ] (directory: String): DStream[(K, V)] = { + ] (directory: String): InputDStream[(K, V)] = { new FileInputDStream[K, V, F](this, directory) } @@ -330,7 +340,7 @@ class StreamingContext private[streaming] ( K: ClassTag, V: ClassTag, F <: NewInputFormat[K, V]: ClassTag - ] (directory: String, filter: Path => Boolean, newFilesOnly: Boolean): DStream[(K, V)] = { + ] (directory: String, filter: Path => Boolean, newFilesOnly: Boolean): InputDStream[(K, V)] = { new FileInputDStream[K, V, F](this, directory, filter, newFilesOnly) } @@ -356,7 +366,7 @@ class StreamingContext private[streaming] ( def queueStream[T: ClassTag]( queue: Queue[RDD[T]], oneAtATime: Boolean = true - ): DStream[T] = { + ): InputDStream[T] = { queueStream(queue, oneAtATime, sc.makeRDD(Seq[T](), 1)) } @@ -373,7 +383,7 @@ class StreamingContext private[streaming] ( queue: Queue[RDD[T]], oneAtATime: Boolean, defaultRDD: RDD[T] - ): DStream[T] = { + ): InputDStream[T] = { new QueueInputDStream(this, queue, oneAtATime, defaultRDD) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala index 13e2bacc92edc..505e4431e4350 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala @@ -97,6 +97,10 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classTag: ClassTag[T] } object JavaDStream { + /** + * Convert a scala [[org.apache.spark.streaming.dstream.DStream]] to a Java-friendly + * [[org.apache.spark.streaming.api.java.JavaDStream]]. + */ implicit def fromDStream[T: ClassTag](dstream: DStream[T]): JavaDStream[T] = new JavaDStream[T](dstream) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala new file mode 100644 index 0000000000000..91f8d342d2bf9 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.api.java + +import scala.reflect.ClassTag + +import org.apache.spark.streaming.dstream.InputDStream + +/** + * A Java-friendly interface to [[org.apache.spark.streaming.dstream.InputDStream]]. + */ +class JavaInputDStream[T](val inputDStream: InputDStream[T]) + (implicit override val classTag: ClassTag[T]) extends JavaDStream[T](inputDStream) { +} + +object JavaInputDStream { + /** + * Convert a scala [[org.apache.spark.streaming.dstream.InputDStream]] to a Java-friendly + * [[org.apache.spark.streaming.api.java.JavaInputDStream]]. + */ + implicit def fromInputDStream[T: ClassTag]( + inputDStream: InputDStream[T]): JavaInputDStream[T] = { + new JavaInputDStream[T](inputDStream) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala new file mode 100644 index 0000000000000..add858530862b --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.api.java + +import org.apache.spark.streaming.dstream.InputDStream +import scala.reflect.ClassTag + +/** + * A Java-friendly interface to [[org.apache.spark.streaming.dstream.InputDStream]] of + * key-value pairs. + */ +class JavaPairInputDStream[K, V](val inputDStream: InputDStream[(K, V)])( + implicit val kClassTag: ClassTag[K], implicit val vClassTag: ClassTag[V] + ) extends JavaPairDStream[K, V](inputDStream) { +} + +object JavaPairInputDStream { + /** + * Convert a scala [[org.apache.spark.streaming.dstream.InputDStream]] of pairs to a + * Java-friendly [[org.apache.spark.streaming.api.java.JavaPairInputDStream]]. + */ + implicit def fromInputDStream[K: ClassTag, V: ClassTag]( + inputDStream: InputDStream[(K, V)]): JavaPairInputDStream[K, V] = { + new JavaPairInputDStream[K, V](inputDStream) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala new file mode 100644 index 0000000000000..974b3e451642d --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.api.java + +import scala.reflect.ClassTag + +import org.apache.spark.streaming.dstream.ReceiverInputDStream + +/** + * A Java-friendly interface to [[org.apache.spark.streaming.dstream.ReceiverInputDStream]], the + * abstract class for defining any input stream that receives data over the network. + */ +class JavaPairReceiverInputDStream[K, V](val receiverInputDStream: ReceiverInputDStream[(K, V)]) + (implicit override val kClassTag: ClassTag[K], override implicit val vClassTag: ClassTag[V]) + extends JavaPairInputDStream[K, V](receiverInputDStream) { +} + +object JavaPairReceiverInputDStream { + /** + * Convert a scala [[org.apache.spark.streaming.dstream.ReceiverInputDStream]] to a Java-friendly + * [[org.apache.spark.streaming.api.java.JavaReceiverInputDStream]]. + */ + implicit def fromReceiverInputDStream[K: ClassTag, V: ClassTag]( + receiverInputDStream: ReceiverInputDStream[(K, V)]): JavaPairReceiverInputDStream[K, V] = { + new JavaPairReceiverInputDStream[K, V](receiverInputDStream) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala new file mode 100644 index 0000000000000..340ef979808b9 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.api.java + +import scala.reflect.ClassTag + +import org.apache.spark.streaming.dstream.ReceiverInputDStream + +/** + * A Java-friendly interface to [[org.apache.spark.streaming.dstream.ReceiverInputDStream]], the + * abstract class for defining any input stream that receives data over the network. + */ +class JavaReceiverInputDStream[T](val receiverInputDStream: ReceiverInputDStream[T]) + (implicit override val classTag: ClassTag[T]) extends JavaInputDStream[T](receiverInputDStream) { +} + +object JavaReceiverInputDStream { + /** + * Convert a scala [[org.apache.spark.streaming.dstream.ReceiverInputDStream]] to a Java-friendly + * [[org.apache.spark.streaming.api.java.JavaReceiverInputDStream]]. + */ + implicit def fromReceiverInputDStream[T: ClassTag]( + receiverInputDStream: ReceiverInputDStream[T]): JavaReceiverInputDStream[T] = { + new JavaReceiverInputDStream[T](receiverInputDStream) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index c800602d0959b..fbb2e9f85dd12 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -35,7 +35,8 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.streaming.scheduler.StreamingListener import org.apache.hadoop.conf.Configuration -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.dstream.{PluggableInputDStream, ReceiverInputDStream, DStream} +import org.apache.spark.streaming.receiver.Receiver /** * A Java-friendly version of [[org.apache.spark.streaming.StreamingContext]] which is the main @@ -155,8 +156,10 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param port Port to connect to for receiving data * @param storageLevel Storage level to use for storing the received objects */ - def socketTextStream(hostname: String, port: Int, storageLevel: StorageLevel) - : JavaDStream[String] = { + def socketTextStream( + hostname: String, port: Int, + storageLevel: StorageLevel + ): JavaReceiverInputDStream[String] = { ssc.socketTextStream(hostname, port, storageLevel) } @@ -167,7 +170,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param hostname Hostname to connect to for receiving data * @param port Port to connect to for receiving data */ - def socketTextStream(hostname: String, port: Int): JavaDStream[String] = { + def socketTextStream(hostname: String, port: Int): JavaReceiverInputDStream[String] = { ssc.socketTextStream(hostname, port) } @@ -186,7 +189,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { port: Int, converter: JFunction[InputStream, java.lang.Iterable[T]], storageLevel: StorageLevel) - : JavaDStream[T] = { + : JavaReceiverInputDStream[T] = { def fn = (x: InputStream) => converter.call(x).toIterator implicit val cmt: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] @@ -218,10 +221,11 @@ class JavaStreamingContext(val ssc: StreamingContext) { def rawSocketStream[T]( hostname: String, port: Int, - storageLevel: StorageLevel): JavaDStream[T] = { + storageLevel: StorageLevel): JavaReceiverInputDStream[T] = { implicit val cmt: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port, storageLevel)) + JavaReceiverInputDStream.fromReceiverInputDStream( + ssc.rawSocketStream(hostname, port, storageLevel)) } /** @@ -233,10 +237,11 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param port Port to connect to for receiving data * @tparam T Type of the objects in the received blocks */ - def rawSocketStream[T](hostname: String, port: Int): JavaDStream[T] = { + def rawSocketStream[T](hostname: String, port: Int): JavaReceiverInputDStream[T] = { implicit val cmt: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port)) + JavaReceiverInputDStream.fromReceiverInputDStream( + ssc.rawSocketStream(hostname, port)) } /** @@ -249,7 +254,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @tparam V Value type for reading HDFS file * @tparam F Input format for reading HDFS file */ - def fileStream[K, V, F <: NewInputFormat[K, V]](directory: String): JavaPairDStream[K, V] = { + def fileStream[K, V, F <: NewInputFormat[K, V]]( + directory: String): JavaPairInputDStream[K, V] = { implicit val cmk: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] implicit val cmv: ClassTag[V] = @@ -275,7 +281,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { name: String, storageLevel: StorageLevel, supervisorStrategy: SupervisorStrategy - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.actorStream[T](props, name, storageLevel, supervisorStrategy) @@ -296,7 +302,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { props: Props, name: String, storageLevel: StorageLevel - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.actorStream[T](props, name, storageLevel) @@ -316,14 +322,14 @@ class JavaStreamingContext(val ssc: StreamingContext) { def actorStream[T]( props: Props, name: String - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.actorStream[T](props, name) } /** - * Creates an input stream from an queue of RDDs. In each batch, + * Create an input stream from an queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * * NOTE: changes to the queue after the stream is created will not be recognized. @@ -339,7 +345,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Creates an input stream from an queue of RDDs. In each batch, + * Create an input stream from an queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * * NOTE: changes to the queue after the stream is created will not be recognized. @@ -347,7 +353,10 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval * @tparam T Type of objects in the RDD */ - def queueStream[T](queue: java.util.Queue[JavaRDD[T]], oneAtATime: Boolean): JavaDStream[T] = { + def queueStream[T]( + queue: java.util.Queue[JavaRDD[T]], + oneAtATime: Boolean + ): JavaInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val sQueue = new scala.collection.mutable.Queue[RDD[T]] @@ -356,7 +365,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Creates an input stream from an queue of RDDs. In each batch, + * Create an input stream from an queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * * NOTE: changes to the queue after the stream is created will not be recognized. @@ -368,7 +377,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { def queueStream[T]( queue: java.util.Queue[JavaRDD[T]], oneAtATime: Boolean, - defaultRDD: JavaRDD[T]): JavaDStream[T] = { + defaultRDD: JavaRDD[T]): JavaInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val sQueue = new scala.collection.mutable.Queue[RDD[T]] @@ -376,6 +385,17 @@ class JavaStreamingContext(val ssc: StreamingContext) { ssc.queueStream(sQueue, oneAtATime, defaultRDD.rdd) } + /** + * Create an input stream with any arbitrary user implemented receiver. + * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html + * @param receiver Custom implementation of Receiver + */ + def receiverStream[T](receiver: Receiver[T]): ReceiverInputDStream[T] = { + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + ssc.receiverStream(receiver) + } + /** * Create a unified DStream from multiple DStreams of the same type and same slide duration. */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala index 226844c2284e3..aa1993f0580a8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala @@ -30,7 +30,7 @@ import scala.reflect.ClassTag * FileInputDStream, a subclass of InputDStream, monitors a HDFS directory from the driver for * new files and generates RDDs with the new files. For implementing input streams * that requires running a receiver on the worker nodes, use - * [[org.apache.spark.streaming.dstream.NetworkInputDStream]] as the parent class. + * [[org.apache.spark.streaming.dstream.ReceiverInputDStream]] as the parent class. * * @param ssc_ Streaming context that will execute this input stream */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala deleted file mode 100644 index 5a249706b4d2f..0000000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ /dev/null @@ -1,362 +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.streaming.dstream - -import java.nio.ByteBuffer -import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} - -import scala.collection.mutable.{ArrayBuffer, HashMap} -import scala.concurrent.Await -import scala.reflect.ClassTag - -import akka.actor.{Actor, Props} -import akka.pattern.ask - -import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.rdd.{BlockRDD, RDD} -import org.apache.spark.storage.{BlockId, StorageLevel, StreamBlockId} -import org.apache.spark.streaming._ -import org.apache.spark.streaming.scheduler.{AddBlock, DeregisterReceiver, ReceivedBlockInfo, RegisterReceiver} -import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} -import org.apache.spark.util.{AkkaUtils, Utils} - -/** - * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] - * that has to start a receiver on worker nodes to receive external data. - * Specific implementations of NetworkInputDStream must - * define the getReceiver() function that gets the receiver object of type - * [[org.apache.spark.streaming.dstream.NetworkReceiver]] that will be sent - * to the workers to receive data. - * @param ssc_ Streaming context that will execute this input stream - * @tparam T Class type of the object of this stream - */ -abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingContext) - extends InputDStream[T](ssc_) { - - /** Keeps all received blocks information */ - private lazy val receivedBlockInfo = new HashMap[Time, Array[ReceivedBlockInfo]] - - /** This is an unique identifier for the network input stream. */ - val id = ssc.getNewNetworkStreamId() - - /** - * Gets the receiver object that will be sent to the worker nodes - * to receive data. This method needs to defined by any specific implementation - * of a NetworkInputDStream. - */ - def getReceiver(): NetworkReceiver[T] - - // Nothing to start or stop as both taken care of by the NetworkInputTracker. - def start() {} - - def stop() {} - - /** Ask NetworkInputTracker for received data blocks and generates RDDs with them. */ - override def compute(validTime: Time): Option[RDD[T]] = { - // If this is called for any time before the start time of the context, - // then this returns an empty RDD. This may happen when recovering from a - // master failure - if (validTime >= graph.startTime) { - val blockInfo = ssc.scheduler.networkInputTracker.getReceivedBlockInfo(id) - receivedBlockInfo(validTime) = blockInfo - val blockIds = blockInfo.map(_.blockId.asInstanceOf[BlockId]) - Some(new BlockRDD[T](ssc.sc, blockIds)) - } else { - Some(new BlockRDD[T](ssc.sc, Array[BlockId]())) - } - } - - /** Get information on received blocks. */ - private[streaming] def getReceivedBlockInfo(time: Time) = { - receivedBlockInfo(time) - } - - /** - * Clear metadata that are older than `rememberDuration` of this DStream. - * This is an internal method that should not be called directly. This - * implementation overrides the default implementation to clear received - * block information. - */ - private[streaming] override def clearMetadata(time: Time) { - super.clearMetadata(time) - val oldReceivedBlocks = receivedBlockInfo.filter(_._1 <= (time - rememberDuration)) - receivedBlockInfo --= oldReceivedBlocks.keys - logDebug("Cleared " + oldReceivedBlocks.size + " RDDs that were older than " + - (time - rememberDuration) + ": " + oldReceivedBlocks.keys.mkString(", ")) - } -} - - -private[streaming] sealed trait NetworkReceiverMessage -private[streaming] case class StopReceiver(msg: String) extends NetworkReceiverMessage - -/** - * Abstract class of a receiver that can be run on worker nodes to receive external data. See - * [[org.apache.spark.streaming.dstream.NetworkInputDStream]] for an explanation. - */ -abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging { - - /** Local SparkEnv */ - lazy protected val env = SparkEnv.get - - /** Remote Akka actor for the NetworkInputTracker */ - lazy protected val trackerActor = { - val ip = env.conf.get("spark.driver.host", "localhost") - val port = env.conf.getInt("spark.driver.port", 7077) - val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) - env.actorSystem.actorSelection(url) - } - - /** Akka actor for receiving messages from the NetworkInputTracker in the driver */ - lazy protected val actor = env.actorSystem.actorOf( - Props(new NetworkReceiverActor()), "NetworkReceiver-" + streamId) - - /** Timeout for Akka actor messages */ - lazy protected val askTimeout = AkkaUtils.askTimeout(env.conf) - - /** Thread that starts the receiver and stays blocked while data is being received */ - lazy protected val receivingThread = Thread.currentThread() - - /** Exceptions that occurs while receiving data */ - protected lazy val exceptions = new ArrayBuffer[Exception] - - /** Identifier of the stream this receiver is associated with */ - protected var streamId: Int = -1 - - /** - * This method will be called to start receiving data. All your receiver - * starting code should be implemented by defining this function. - */ - protected def onStart() - - /** This method will be called to stop receiving data. */ - protected def onStop() - - /** Conveys a placement preference (hostname) for this receiver. */ - def getLocationPreference() : Option[String] = None - - /** - * Start the receiver. First is accesses all the lazy members to - * materialize them. Then it calls the user-defined onStart() method to start - * other threads, etc required to receiver the data. - */ - def start() { - try { - // Access the lazy vals to materialize them - env - actor - receivingThread - - // Call user-defined onStart() - logInfo("Starting receiver") - onStart() - - // Wait until interrupt is called on this thread - while(true) Thread.sleep(100000) - } catch { - case ie: InterruptedException => - logInfo("Receiving thread has been interrupted, receiver " + streamId + " stopped") - case e: Exception => - logError("Error receiving data in receiver " + streamId, e) - exceptions += e - } - - // Call user-defined onStop() - logInfo("Stopping receiver") - try { - onStop() - } catch { - case e: Exception => - logError("Error stopping receiver " + streamId, e) - exceptions += e - } - - val message = if (exceptions.isEmpty) { - null - } else if (exceptions.size == 1) { - val e = exceptions.head - "Exception in receiver " + streamId + ": " + e.getMessage + "\n" + e.getStackTraceString - } else { - "Multiple exceptions in receiver " + streamId + "(" + exceptions.size + "):\n" - exceptions.zipWithIndex.map { - case (e, i) => "Exception " + i + ": " + e.getMessage + "\n" + e.getStackTraceString - }.mkString("\n") - } - - logInfo("Deregistering receiver " + streamId) - val future = trackerActor.ask(DeregisterReceiver(streamId, message))(askTimeout) - Await.result(future, askTimeout) - logInfo("Deregistered receiver " + streamId) - env.actorSystem.stop(actor) - logInfo("Stopped receiver " + streamId) - } - - /** - * Stop the receiver. First it interrupts the main receiving thread, - * that is, the thread that called receiver.start(). - */ - def stop() { - // Stop receiving by interrupting the receiving thread - receivingThread.interrupt() - logInfo("Interrupted receiving thread " + receivingThread + " for stopping") - } - - /** - * Stop the receiver and reports exception to the tracker. - * This should be called whenever an exception is to be handled on any thread - * of the receiver. - */ - protected def stopOnError(e: Exception) { - logError("Error receiving data", e) - exceptions += e - stop() - } - - /** - * Push a block (as an ArrayBuffer filled with data) into the block manager. - */ - def pushBlock( - blockId: StreamBlockId, - arrayBuffer: ArrayBuffer[T], - metadata: Any, - level: StorageLevel - ) { - env.blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], level) - trackerActor ! AddBlock(ReceivedBlockInfo(streamId, blockId, arrayBuffer.size, metadata)) - logDebug("Pushed block " + blockId) - } - - /** - * Push a block (as bytes) into the block manager. - */ - def pushBlock( - blockId: StreamBlockId, - bytes: ByteBuffer, - metadata: Any, - level: StorageLevel - ) { - env.blockManager.putBytes(blockId, bytes, level) - trackerActor ! AddBlock(ReceivedBlockInfo(streamId, blockId, -1, metadata)) - } - - /** Set the ID of the DStream that this receiver is associated with */ - protected[streaming] def setStreamId(id: Int) { - streamId = id - } - - /** A helper actor that communicates with the NetworkInputTracker */ - private class NetworkReceiverActor extends Actor { - - override def preStart() { - val msg = RegisterReceiver( - streamId, NetworkReceiver.this.getClass.getSimpleName, Utils.localHostName(), self) - val future = trackerActor.ask(msg)(askTimeout) - Await.result(future, askTimeout) - logInfo("Registered receiver " + streamId) - } - - override def receive() = { - case StopReceiver => - logInfo("Received stop signal") - stop() - } - } - - /** - * Batches objects created by a [[org.apache.spark.streaming.dstream.NetworkReceiver]] and puts - * them into appropriately named blocks at regular intervals. This class starts two threads, - * one to periodically start a new batch and prepare the previous batch of as a block, - * the other to push the blocks into the block manager. - */ - class BlockGenerator(storageLevel: StorageLevel) - extends Serializable with Logging { - - case class Block(id: StreamBlockId, buffer: ArrayBuffer[T], metadata: Any = null) - - val clock = new SystemClock() - val blockInterval = env.conf.getLong("spark.streaming.blockInterval", 200) - val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer, - "BlockGenerator") - val blockStorageLevel = storageLevel - val blocksForPushing = new ArrayBlockingQueue[Block](1000) - val blockPushingThread = new Thread() { override def run() { keepPushingBlocks() } } - - var currentBuffer = new ArrayBuffer[T] - var stopped = false - - def start() { - blockIntervalTimer.start() - blockPushingThread.start() - logInfo("Started BlockGenerator") - } - - def stop() { - blockIntervalTimer.stop(false) - stopped = true - blockPushingThread.join() - logInfo("Stopped BlockGenerator") - } - - def += (obj: T): Unit = synchronized { - currentBuffer += obj - } - - private def updateCurrentBuffer(time: Long): Unit = synchronized { - try { - val newBlockBuffer = currentBuffer - currentBuffer = new ArrayBuffer[T] - if (newBlockBuffer.size > 0) { - val blockId = StreamBlockId(NetworkReceiver.this.streamId, time - blockInterval) - val newBlock = new Block(blockId, newBlockBuffer) - blocksForPushing.add(newBlock) - } - } catch { - case ie: InterruptedException => - logInfo("Block updating timer thread was interrupted") - case e: Exception => - NetworkReceiver.this.stopOnError(e) - } - } - - private def keepPushingBlocks() { - logInfo("Started block pushing thread") - try { - while(!stopped) { - Option(blocksForPushing.poll(100, TimeUnit.MILLISECONDS)) match { - case Some(block) => - NetworkReceiver.this.pushBlock(block.id, block.buffer, block.metadata, storageLevel) - case None => - } - } - // Push out the blocks that are still left - logInfo("Pushing out the last " + blocksForPushing.size() + " blocks") - while (!blocksForPushing.isEmpty) { - val block = blocksForPushing.take() - NetworkReceiver.this.pushBlock(block.id, block.buffer, block.metadata, storageLevel) - logInfo("Blocks left to push " + blocksForPushing.size()) - } - logInfo("Stopped blocks pushing thread") - } catch { - case ie: InterruptedException => - logInfo("Block pushing thread was interrupted") - case e: Exception => - NetworkReceiver.this.stopOnError(e) - } - } - } -} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala index 6f9477020a459..186e1bf03a944 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala @@ -19,13 +19,14 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.StreamingContext import scala.reflect.ClassTag +import org.apache.spark.streaming.receiver.Receiver private[streaming] class PluggableInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, - receiver: NetworkReceiver[T]) extends NetworkInputDStream[T](ssc_) { + receiver: Receiver[T]) extends ReceiverInputDStream[T](ssc_) { - def getReceiver(): NetworkReceiver[T] = { + def getReceiver(): Receiver[T] = { receiver } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala index dea0f26f908fb..e2925b9e03ec3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.storage.{StorageLevel, StreamBlockId} import org.apache.spark.streaming.StreamingContext @@ -28,6 +28,7 @@ import java.nio.ByteBuffer import java.nio.channels.{ReadableByteChannel, SocketChannel} import java.io.EOFException import java.util.concurrent.ArrayBlockingQueue +import org.apache.spark.streaming.receiver.Receiver /** @@ -42,21 +43,19 @@ class RawInputDStream[T: ClassTag]( host: String, port: Int, storageLevel: StorageLevel - ) extends NetworkInputDStream[T](ssc_ ) with Logging { + ) extends ReceiverInputDStream[T](ssc_ ) with Logging { - def getReceiver(): NetworkReceiver[T] = { - new RawNetworkReceiver(host, port, storageLevel).asInstanceOf[NetworkReceiver[T]] + def getReceiver(): Receiver[T] = { + new RawNetworkReceiver(host, port, storageLevel).asInstanceOf[Receiver[T]] } } private[streaming] class RawNetworkReceiver(host: String, port: Int, storageLevel: StorageLevel) - extends NetworkReceiver[Any] { + extends Receiver[Any](storageLevel) with Logging { var blockPushingThread: Thread = null - override def getLocationPreference = None - def onStart() { // Open a socket to the target address and keep reading from it logInfo("Connecting to " + host + ":" + port) @@ -73,9 +72,8 @@ class RawNetworkReceiver(host: String, port: Int, storageLevel: StorageLevel) var nextBlockNumber = 0 while (true) { val buffer = queue.take() - val blockId = StreamBlockId(streamId, nextBlockNumber) nextBlockNumber += 1 - pushBlock(blockId, buffer, null, storageLevel) + store(buffer) } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala new file mode 100644 index 0000000000000..75cabdbf8da26 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.dstream + +import scala.collection.mutable.HashMap +import scala.reflect.ClassTag + +import org.apache.spark.rdd.{BlockRDD, RDD} +import org.apache.spark.storage.BlockId +import org.apache.spark.streaming._ +import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.streaming.scheduler.ReceivedBlockInfo + +/** + * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] + * that has to start a receiver on worker nodes to receive external data. + * Specific implementations of NetworkInputDStream must + * define `the getReceiver()` function that gets the receiver object of type + * [[org.apache.spark.streaming.receiver.Receiver]] that will be sent + * to the workers to receive data. + * @param ssc_ Streaming context that will execute this input stream + * @tparam T Class type of the object of this stream + */ +abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingContext) + extends InputDStream[T](ssc_) { + + /** Keeps all received blocks information */ + private lazy val receivedBlockInfo = new HashMap[Time, Array[ReceivedBlockInfo]] + + /** This is an unique identifier for the network input stream. */ + val id = ssc.getNewReceiverStreamId() + + /** + * Gets the receiver object that will be sent to the worker nodes + * to receive data. This method needs to defined by any specific implementation + * of a NetworkInputDStream. + */ + def getReceiver(): Receiver[T] + + // Nothing to start or stop as both taken care of by the ReceiverInputTracker. + def start() {} + + def stop() {} + + /** Ask ReceiverInputTracker for received data blocks and generates RDDs with them. */ + override def compute(validTime: Time): Option[RDD[T]] = { + // If this is called for any time before the start time of the context, + // then this returns an empty RDD. This may happen when recovering from a + // master failure + if (validTime >= graph.startTime) { + val blockInfo = ssc.scheduler.receiverTracker.getReceivedBlockInfo(id) + receivedBlockInfo(validTime) = blockInfo + val blockIds = blockInfo.map(_.blockId.asInstanceOf[BlockId]) + Some(new BlockRDD[T](ssc.sc, blockIds)) + } else { + Some(new BlockRDD[T](ssc.sc, Array[BlockId]())) + } + } + + /** Get information on received blocks. */ + private[streaming] def getReceivedBlockInfo(time: Time) = { + receivedBlockInfo(time) + } + + /** + * Clear metadata that are older than `rememberDuration` of this DStream. + * This is an internal method that should not be called directly. This + * implementation overrides the default implementation to clear received + * block information. + */ + private[streaming] override def clearMetadata(time: Time) { + super.clearMetadata(time) + val oldReceivedBlocks = receivedBlockInfo.filter(_._1 <= (time - rememberDuration)) + receivedBlockInfo --= oldReceivedBlocks.keys + logDebug("Cleared " + oldReceivedBlocks.size + " RDDs that were older than " + + (time - rememberDuration) + ": " + oldReceivedBlocks.keys.mkString(", ")) + } +} + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala index 63d94d1cc670a..1e32727eacfa3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala @@ -24,7 +24,9 @@ import org.apache.spark.util.NextIterator import scala.reflect.ClassTag import java.io._ -import java.net.Socket +import java.net.{UnknownHostException, Socket} +import org.apache.spark.Logging +import org.apache.spark.streaming.receiver.Receiver private[streaming] class SocketInputDStream[T: ClassTag]( @@ -33,9 +35,9 @@ class SocketInputDStream[T: ClassTag]( port: Int, bytesToObjects: InputStream => Iterator[T], storageLevel: StorageLevel - ) extends NetworkInputDStream[T](ssc_) { + ) extends ReceiverInputDStream[T](ssc_) { - def getReceiver(): NetworkReceiver[T] = { + def getReceiver(): Receiver[T] = { new SocketReceiver(host, port, bytesToObjects, storageLevel) } } @@ -46,26 +48,52 @@ class SocketReceiver[T: ClassTag]( port: Int, bytesToObjects: InputStream => Iterator[T], storageLevel: StorageLevel - ) extends NetworkReceiver[T] { + ) extends Receiver[T](storageLevel) with Logging { - lazy protected val blockGenerator = new BlockGenerator(storageLevel) + var socket: Socket = null + var receivingThread: Thread = null - override def getLocationPreference = None + def onStart() { + receivingThread = new Thread("Socket Receiver") { + override def run() { + connect() + receive() + } + } + receivingThread.start() + } - protected def onStart() { - logInfo("Connecting to " + host + ":" + port) - val socket = new Socket(host, port) - logInfo("Connected to " + host + ":" + port) - blockGenerator.start() - val iterator = bytesToObjects(socket.getInputStream()) - while(iterator.hasNext) { - val obj = iterator.next - blockGenerator += obj + def onStop() { + if (socket != null) { + socket.close() + } + socket = null + if (receivingThread != null) { + receivingThread.join() } } - protected def onStop() { - blockGenerator.stop() + def connect() { + try { + logInfo("Connecting to " + host + ":" + port) + socket = new Socket(host, port) + } catch { + case e: Exception => + restart("Could not connect to " + host + ":" + port, e) + } + } + + def receive() { + try { + logInfo("Connected to " + host + ":" + port) + val iterator = bytesToObjects(socket.getInputStream()) + while(!isStopped && iterator.hasNext) { + store(iterator.next) + } + } catch { + case e: Exception => + restart("Error receiving data from socket", e) + } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala similarity index 66% rename from streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala rename to streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala index da0d364ae7bdb..821cf19481d44 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala @@ -15,26 +15,22 @@ * limitations under the License. */ -package org.apache.spark.streaming.receivers +package org.apache.spark.streaming.receiver -import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy } -import akka.actor.{ actorRef2Scala, ActorRef } -import akka.actor.{ PossiblyHarmful, OneForOneStrategy } -import akka.actor.SupervisorStrategy._ +import java.util.concurrent.atomic.AtomicInteger import scala.concurrent.duration._ import scala.language.postfixOps import scala.reflect.ClassTag -import org.apache.spark.storage.{StorageLevel, StreamBlockId} -import org.apache.spark.streaming.dstream.NetworkReceiver - -import java.util.concurrent.atomic.AtomicInteger - -import scala.collection.mutable.ArrayBuffer +import akka.actor._ +import akka.actor.SupervisorStrategy.{Escalate, Restart} +import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.storage.StorageLevel +import java.nio.ByteBuffer /** A helper with set of defaults for supervisor strategy */ -object ReceiverSupervisorStrategy { +object ActorSupervisorStrategy { val defaultStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange = 15 millis) { @@ -50,9 +46,9 @@ object ReceiverSupervisorStrategy { * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html * * @example {{{ - * class MyActor extends Actor with Receiver{ + * class MyActor extends Actor with ActorHelper{ * def receive { - * case anything: String => pushBlock(anything) + * case anything: String => store(anything) * } * } * @@ -65,29 +61,40 @@ object ReceiverSupervisorStrategy { * to ensure the type safety, i.e parametrized type of push block and InputDStream * should be same. */ -trait Receiver { +trait ActorHelper { self: Actor => // to ensure that this can be added to Actor classes only + /** Store an iterator of received data as a data block into Spark's memory. */ + def store[T](iter: Iterator[T]) { + println("Storing iterator") + context.parent ! IteratorData(iter) + } + /** - * Push an iterator received data into Spark Streaming for processing + * Store the bytes of received data as a data block into Spark's memory. Note + * that the data in the ByteBuffer must be serialized using the same serializer + * that Spark is configured to use. */ - def pushBlock[T: ClassTag](iter: Iterator[T]) { - context.parent ! Data(iter) + def store(bytes: ByteBuffer) { + context.parent ! ByteBufferData(bytes) } /** - * Push a single item of received data into Spark Streaming for processing + * Store a single item of received data to Spark's memory. + * These single items will be aggregated together into data blocks before + * being pushed into Spark's memory. */ - def pushBlock[T: ClassTag](data: T) { - context.parent ! Data(data) + def store[T](item: T) { + println("Storing item") + context.parent ! SingleItemData(item) } } /** * Statistics for querying the supervisor about state of workers. Used in * conjunction with `StreamingContext.actorStream` and - * [[org.apache.spark.streaming.receivers.Receiver]]. + * [[org.apache.spark.streaming.receiver.ActorHelper]]. */ case class Statistics(numberOfMsgs: Int, numberOfWorkers: Int, @@ -95,7 +102,10 @@ case class Statistics(numberOfMsgs: Int, otherInfo: String) /** Case class to receive data sent by child actors */ -private[streaming] case class Data[T: ClassTag](data: T) +private[streaming] sealed trait ActorReceiverData +private[streaming] case class SingleItemData[T](item: T) extends ActorReceiverData +private[streaming] case class IteratorData[T](iterator: Iterator[T]) extends ActorReceiverData +private[streaming] case class ByteBufferData(bytes: ByteBuffer) extends ActorReceiverData /** * Provides Actors as receivers for receiving stream. @@ -117,16 +127,13 @@ private[streaming] case class Data[T: ClassTag](data: T) * }}} */ private[streaming] class ActorReceiver[T: ClassTag]( - props: Props, - name: String, - storageLevel: StorageLevel, - receiverSupervisorStrategy: SupervisorStrategy) - extends NetworkReceiver[T] { + props: Props, + name: String, + storageLevel: StorageLevel, + receiverSupervisorStrategy: SupervisorStrategy + ) extends Receiver[T](storageLevel) with Logging { - protected lazy val blocksGenerator: BlockGenerator = - new BlockGenerator(storageLevel) - - protected lazy val supervisor = env.actorSystem.actorOf(Props(new Supervisor), + protected lazy val supervisor = SparkEnv.get.actorSystem.actorOf(Props(new Supervisor), "Supervisor" + streamId) class Supervisor extends Actor { @@ -140,12 +147,18 @@ private[streaming] class ActorReceiver[T: ClassTag]( def receive = { - case Data(iter: Iterator[_]) => pushBlock(iter.asInstanceOf[Iterator[T]]) + case IteratorData(iterator) => + println("received iterator") + store(iterator.asInstanceOf[Iterator[T]]) - case Data(msg) => - blocksGenerator += msg.asInstanceOf[T] + case SingleItemData(msg) => + println("received single") + store(msg.asInstanceOf[T]) n.incrementAndGet + case ByteBufferData(bytes) => + store(bytes) + case props: Props => val worker = context.actorOf(props) logInfo("Started receiver worker at:" + worker.path) @@ -165,20 +178,14 @@ private[streaming] class ActorReceiver[T: ClassTag]( } } - protected def pushBlock(iter: Iterator[T]) { - val buffer = new ArrayBuffer[T] - buffer ++= iter - pushBlock(StreamBlockId(streamId, System.nanoTime()), buffer, null, storageLevel) - } - - protected def onStart() = { - blocksGenerator.start() + def onStart() = { supervisor logInfo("Supervision tree for receivers initialized at:" + supervisor.path) } - protected def onStop() = { + def onStop() = { supervisor ! PoisonPill } } + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala new file mode 100644 index 0000000000000..78cc2daa56e53 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.receiver + +import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.storage.StreamBlockId +import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} + +/** Listener object for BlockGenerator events */ +private[streaming] trait BlockGeneratorListener { + /** Called when a new block needs to be pushed */ + def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) + /** Called when an error has occurred in BlockGenerator */ + def onError(message: String, throwable: Throwable) +} + +/** + * Generates batches of objects received by a + * [[org.apache.spark.streaming.receiver.Receiver]] and puts them into appropriately + * named blocks at regular intervals. This class starts two threads, + * one to periodically start a new batch and prepare the previous batch of as a block, + * the other to push the blocks into the block manager. + */ +private[streaming] class BlockGenerator( + listener: BlockGeneratorListener, + receiverId: Int, + conf: SparkConf + ) extends Logging { + + private case class Block(id: StreamBlockId, buffer: ArrayBuffer[Any]) + + private val clock = new SystemClock() + private val blockInterval = conf.getLong("spark.streaming.blockInterval", 200) + private val blockIntervalTimer = + new RecurringTimer(clock, blockInterval, updateCurrentBuffer, "BlockGenerator") + private val blockQueueSize = conf.getInt("spark.streaming.blockQueueSize", 10) + private val blocksForPushing = new ArrayBlockingQueue[Block](blockQueueSize) + private val blockPushingThread = new Thread() { override def run() { keepPushingBlocks() } } + + @volatile private var currentBuffer = new ArrayBuffer[Any] + @volatile private var stopped = false + + /** Start block generating and pushing threads. */ + def start() { + blockIntervalTimer.start() + blockPushingThread.start() + logInfo("Started BlockGenerator") + } + + /** Stop all threads. */ + def stop() { + logInfo("Stopping BlockGenerator") + blockIntervalTimer.stop(interruptTimer = false) + stopped = true + logInfo("Waiting for block pushing thread") + blockPushingThread.join() + logInfo("Stopped BlockGenerator") + } + + /** + * Push a single data item into the buffer. All received data items + * will be periodically pushed into BlockManager. + */ + def += (data: Any): Unit = synchronized { + currentBuffer += data + } + + /** Change the buffer to which single records are added to. */ + private def updateCurrentBuffer(time: Long): Unit = synchronized { + try { + val newBlockBuffer = currentBuffer + currentBuffer = new ArrayBuffer[Any] + if (newBlockBuffer.size > 0) { + val blockId = StreamBlockId(receiverId, time - blockInterval) + val newBlock = new Block(blockId, newBlockBuffer) + blocksForPushing.put(newBlock) // put is blocking when queue is full + logDebug("Last element in " + blockId + " is " + newBlockBuffer.last) + } + } catch { + case ie: InterruptedException => + logInfo("Block updating timer thread was interrupted") + case t: Throwable => + reportError("Error in block updating thread", t) + } + } + + /** Keep pushing blocks to the BlockManager. */ + private def keepPushingBlocks() { + logInfo("Started block pushing thread") + try { + while(!stopped) { + Option(blocksForPushing.poll(100, TimeUnit.MILLISECONDS)) match { + case Some(block) => pushBlock(block) + case None => + } + } + // Push out the blocks that are still left + logInfo("Pushing out the last " + blocksForPushing.size() + " blocks") + while (!blocksForPushing.isEmpty) { + logDebug("Getting block ") + val block = blocksForPushing.take() + pushBlock(block) + logInfo("Blocks left to push " + blocksForPushing.size()) + } + logInfo("Stopped block pushing thread") + } catch { + case ie: InterruptedException => + logInfo("Block pushing thread was interrupted") + case t: Throwable => + reportError("Error in block pushing thread", t) + } + } + + private def reportError(message: String, t: Throwable) { + logError(message, t) + listener.onError(message, t) + } + + private def pushBlock(block: Block) { + listener.onPushBlock(block.id, block.buffer) + logInfo("Pushed block " + block.id) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala new file mode 100644 index 0000000000000..44eecf1dd2567 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -0,0 +1,236 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.receiver + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConversions._ + +import org.apache.spark.storage.StorageLevel + +/** + * Abstract class of a receiver that can be run on worker nodes to receive external data. A + * custom receiver can be defined by defining the functions onStart() and onStop(). onStart() + * should define the setup steps necessary to start receiving data, + * and onStop() should define the cleanup steps necessary to stop receiving data. A custom + * receiver would look something like this. + * + * @example {{{ + * class MyReceiver(storageLevel: StorageLevel) extends NetworkReceiver[String](storageLevel) { + * def onStart() { + * // Setup stuff (start threads, open sockets, etc.) to start receiving data. + * // Must start new thread to receive data, as onStart() must be non-blocking. + * + * // Call store(...) in those threads to store received data into Spark's memory. + * + * // Call stop(...), restart() or reportError(...) on any thread based on how + * // different errors should be handled. + * + * // See corresponding method documentation for more details + * } + * + * def onStop() { + * // Cleanup stuff (stop threads, close sockets, etc.) to stop receiving data. + * } + * } + * }}} + */ +abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable { + + /** + * This method is called by the system when the receiver is started. This function + * must initialize all resources (threads, buffers, etc.) necessary for receiving data. + * This function must be non-blocking, so receiving the data must occur on a different + * thread. Received data can be stored with Spark by calling `store(data)`. + * + * If there are errors in threads started here, then following options can be done + * (i) `reportError(...)` can be called to report the error to the driver. + * The receiving of data will continue uninterrupted. + * (ii) `stop(...)` can be called to stop receiving data. This will call `onStop()` to + * clear up all resources allocated (threads, buffers, etc.) during `onStart()`. + * (iii) `restart(...)` can be called to restart the receiver. This will call `onStop()` + * immediately, and then `onStart()` after a delay. + */ + def onStart() + + /** + * This method is called by the system when the receiver is stopped. All resources + * (threads, buffers, etc.) setup in `onStart()` must be cleaned up in this method. + */ + def onStop() + + /** Override this to specify a preferred location (hostname). */ + def preferredLocation : Option[String] = None + + /** + * Store a single item of received data to Spark's memory. + * These single items will be aggregated together into data blocks before + * being pushed into Spark's memory. + */ + def store(dataItem: T) { + executor.pushSingle(dataItem) + } + + /** Store an ArrayBuffer of received data as a data block into Spark's memory. */ + def store(dataBuffer: ArrayBuffer[T]) { + executor.pushArrayBuffer(dataBuffer, None, None) + } + + /** + * Store an ArrayBuffer of received data as a data block into Spark's memory. + * The metadata will be associated with this block of data + * for being used in the corresponding InputDStream. + */ + def store(dataBuffer: ArrayBuffer[T], metadata: Any) { + executor.pushArrayBuffer(dataBuffer, Some(metadata), None) + } + + /** Store an iterator of received data as a data block into Spark's memory. */ + def store(dataIterator: Iterator[T]) { + executor.pushIterator(dataIterator, None, None) + } + + /** + * Store an iterator of received data as a data block into Spark's memory. + * The metadata will be associated with this block of data + * for being used in the corresponding InputDStream. + */ + def store(dataIterator: java.util.Iterator[T], metadata: Any) { + executor.pushIterator(dataIterator, Some(metadata), None) + } + + /** Store an iterator of received data as a data block into Spark's memory. */ + def store(dataIterator: java.util.Iterator[T]) { + executor.pushIterator(dataIterator, None, None) + } + + /** + * Store an iterator of received data as a data block into Spark's memory. + * The metadata will be associated with this block of data + * for being used in the corresponding InputDStream. + */ + def store(dataIterator: Iterator[T], metadata: Any) { + executor.pushIterator(dataIterator, Some(metadata), None) + } + + /** + * Store the bytes of received data as a data block into Spark's memory. Note + * that the data in the ByteBuffer must be serialized using the same serializer + * that Spark is configured to use. + */ + def store(bytes: ByteBuffer) { + executor.pushBytes(bytes, None, None) + } + + /** + * Store the bytes of received data as a data block into Spark's memory. + * The metadata will be associated with this block of data + * for being used in the corresponding InputDStream. + */ + def store(bytes: ByteBuffer, metadata: Any) { + executor.pushBytes(bytes, Some(metadata), None) + } + + /** Report exceptions in receiving data. */ + def reportError(message: String, throwable: Throwable) { + executor.reportError(message, throwable) + } + + /** + * Restart the receiver. This will call `onStop()` immediately and return. + * Asynchronously, after a delay, `onStart()` will be called. + * The `message` will be reported to the driver. + * The delay is defined by the Spark configuration + * `spark.streaming.receiverRestartDelay`. + */ + def restart(message: String) { + executor.restartReceiver(message) + } + + /** + * Restart the receiver. This will call `onStop()` immediately and return. + * Asynchronously, after a delay, `onStart()` will be called. + * The `message` and `exception` will be reported to the driver. + * The delay is defined by the Spark configuration + * `spark.streaming.receiverRestartDelay`. + */ + def restart(message: String, error: Throwable) { + executor.restartReceiver(message, Some(error)) + } + + /** + * Restart the receiver. This will call `onStop()` immediately and return. + * Asynchronously, after the given delay, `onStart()` will be called. + */ + def restart(message: String, error: Throwable, millisecond: Int) { + executor.restartReceiver(message, Some(error), millisecond) + } + + /** Stop the receiver completely. */ + def stop(message: String) { + executor.stop(message, None) + } + + /** Stop the receiver completely due to an exception */ + def stop(message: String, error: Throwable) { + executor.stop(message, Some(error)) + } + + def isStarted(): Boolean = { + executor.isReceiverStarted() + } + + /** Check if receiver has been marked for stopping. */ + def isStopped(): Boolean = { + !executor.isReceiverStarted() + } + + /** Get unique identifier of this receiver. */ + def streamId = id + + /* + * ================= + * Private methods + * ================= + */ + + /** Identifier of the stream this receiver is associated with. */ + private var id: Int = -1 + + /** Handler object that runs the receiver. This is instantiated lazily in the worker. */ + private[streaming] var executor_ : ReceiverSupervisor = null + + /** Set the ID of the DStream that this receiver is associated with. */ + private[streaming] def setReceiverId(id_ : Int) { + id = id_ + } + + /** Attach Network Receiver executor to this receiver. */ + private[streaming] def attachExecutor(exec: ReceiverSupervisor) { + assert(executor_ == null) + executor_ = exec + } + + /** Get the attached executor. */ + private def executor = { + assert(executor_ != null, "Executor has not been attached to this receiver") + executor_ + } +} + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala new file mode 100644 index 0000000000000..6ab3ca6ea5fa6 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.receiver + +/** Messages sent to the NetworkReceiver. */ +private[streaming] sealed trait NetworkReceiverMessage +private[streaming] object StopReceiver extends NetworkReceiverMessage + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala new file mode 100644 index 0000000000000..256b3335e49aa --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.receiver + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.storage.StreamBlockId +import java.util.concurrent.CountDownLatch +import scala.concurrent._ +import ExecutionContext.Implicits.global + +/** + * Abstract class that is responsible for supervising a Receiver in the worker. + * It provides all the necessary interfaces for handling the data received by the receiver. + */ +private[streaming] abstract class ReceiverSupervisor( + receiver: Receiver[_], + conf: SparkConf + ) extends Logging { + + /** Enumeration to identify current state of the StreamingContext */ + object ReceiverState extends Enumeration { + type CheckpointState = Value + val Initialized, Started, Stopped = Value + } + import ReceiverState._ + + // Attach the executor to the receiver + receiver.attachExecutor(this) + + /** Receiver id */ + protected val streamId = receiver.streamId + + /** Has the receiver been marked for stop. */ + private val stopLatch = new CountDownLatch(1) + + /** Time between a receiver is stopped and started again */ + private val defaultRestartDelay = conf.getInt("spark.streaming.receiverRestartDelay", 2000) + + /** Exception associated with the stopping of the receiver */ + @volatile protected var stoppingError: Throwable = null + + /** State of the receiver */ + @volatile private[streaming] var receiverState = Initialized + + /** Push a single data item to backend data store. */ + def pushSingle(data: Any) + + /** Store the bytes of received data as a data block into Spark's memory. */ + def pushBytes( + bytes: ByteBuffer, + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) + + /** Store a iterator of received data as a data block into Spark's memory. */ + def pushIterator( + iterator: Iterator[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) + + /** Store an ArrayBuffer of received data as a data block into Spark's memory. */ + def pushArrayBuffer( + arrayBuffer: ArrayBuffer[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) + + /** Report errors. */ + def reportError(message: String, throwable: Throwable) + + /** Start the executor */ + def start() { + startReceiver() + } + + /** Mark the executor and the receiver for stopping */ + def stop(message: String, error: Option[Throwable]) { + stoppingError = error.orNull + stopReceiver(message, error) + stopLatch.countDown() + } + + /** Start receiver */ + def startReceiver(): Unit = synchronized { + try { + logInfo("Starting receiver") + onReceiverStart() + receiverState = Started + } catch { + case t: Throwable => + stop("Error starting receiver " + streamId, Some(t)) + } + } + + /** Stop receiver */ + def stopReceiver(message: String, error: Option[Throwable]): Unit = synchronized { + try { + receiverState = Stopped + onReceiverStop(message, error) + } catch { + case t: Throwable => + stop("Error stopping receiver " + streamId, Some(t)) + } + } + + /** Restart receiver with delay */ + def restartReceiver(message: String, error: Option[Throwable] = None) { + restartReceiver(message, error, defaultRestartDelay) + } + + /** Restart receiver with delay */ + def restartReceiver(message: String, error: Option[Throwable], delay: Int) { + logWarning("Restarting receiver with delay " + delay + " ms: " + message, + error.getOrElse(null)) + stopReceiver("Restarting receiver with delay " + delay + "ms: " + message, error) + future { + logDebug("Sleeping for " + delay) + Thread.sleep(delay) + logDebug("Starting receiver again") + startReceiver() + logInfo("Receiver started again") + } + } + + /** Called when the receiver needs to be started */ + protected def onReceiverStart(): Unit = synchronized { + // Call user-defined onStart() + logInfo("Calling receiver onStart") + receiver.onStart() + logInfo("Called receiver onStart") + } + + /** Called when the receiver needs to be stopped */ + protected def onReceiverStop(message: String, error: Option[Throwable]): Unit = synchronized { + // Call user-defined onStop() + logInfo("Calling receiver onStop") + receiver.onStop() + logInfo("Called receiver onStop") + } + + /** Check if receiver has been marked for stopping */ + def isReceiverStarted() = { + logDebug("state = " + receiverState) + receiverState == Started + } + + /** Wait the thread until the executor is stopped */ + def awaitTermination() { + stopLatch.await() + logInfo("Waiting for executor stop is over") + if (stoppingError != null) { + logError("Stopped executor with error: " + stoppingError) + } else { + logWarning("Stopped executor without error") + } + if (stoppingError != null) { + throw stoppingError + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala new file mode 100644 index 0000000000000..2a3521bd46ae7 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.receiver + +import java.nio.ByteBuffer +import java.util.concurrent.atomic.AtomicLong + +import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} +import scala.concurrent.Await + +import akka.actor.{Actor, Props} +import akka.pattern.ask + +import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.storage.StreamBlockId +import org.apache.spark.streaming.scheduler._ +import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.storage.StreamBlockId +import org.apache.spark.streaming.scheduler.DeregisterReceiver +import org.apache.spark.streaming.scheduler.AddBlock +import scala.Some +import org.apache.spark.streaming.scheduler.RegisterReceiver +import com.google.common.base.Throwables + +/** + * Concrete implementation of [[org.apache.spark.streaming.receiver.ReceiverSupervisor]] + * which provides all the necessary functionality for handling the data received by + * the receiver. Specifically, it creates a [[org.apache.spark.streaming.receiver.BlockGenerator]] + * object that is used to divide the received data stream into blocks of data. + */ +private[streaming] class ReceiverSupervisorImpl( + receiver: Receiver[_], + env: SparkEnv + ) extends ReceiverSupervisor(receiver, env.conf) with Logging { + + private val blockManager = env.blockManager + + private val storageLevel = receiver.storageLevel + + /** Remote Akka actor for the ReceiverTracker */ + private val trackerActor = { + val ip = env.conf.get("spark.driver.host", "localhost") + val port = env.conf.getInt("spark.driver.port", 7077) + val url = "akka.tcp://spark@%s:%s/user/ReceiverTracker".format(ip, port) + env.actorSystem.actorSelection(url) + } + + /** Timeout for Akka actor messages */ + private val askTimeout = AkkaUtils.askTimeout(env.conf) + + /** Akka actor for receiving messages from the ReceiverTracker in the driver */ + private val actor = env.actorSystem.actorOf( + Props(new Actor { + override def preStart() { + logInfo("Registered receiver " + streamId) + val msg = RegisterReceiver( + streamId, receiver.getClass.getSimpleName, Utils.localHostName(), self) + val future = trackerActor.ask(msg)(askTimeout) + Await.result(future, askTimeout) + } + + override def receive() = { + case StopReceiver => + logInfo("Received stop signal") + stop("Stopped by driver", None) + } + }), "Receiver-" + streamId + "-" + System.currentTimeMillis()) + + /** Unique block ids if one wants to add blocks directly */ + private val newBlockId = new AtomicLong(System.currentTimeMillis()) + + /** Divides received data records into data blocks for pushing in BlockManager. */ + private val blockGenerator = new BlockGenerator(new BlockGeneratorListener { + def onError(message: String, throwable: Throwable) { + reportError(message, throwable) + } + + def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) { + pushArrayBuffer(arrayBuffer, None, Some(blockId)) + } + }, streamId, env.conf) + + /** Push a single record of received data into block generator. */ + def pushSingle(data: Any) { + blockGenerator += (data) + } + + /** Store an ArrayBuffer of received data as a data block into Spark's memory. */ + def pushArrayBuffer( + arrayBuffer: ArrayBuffer[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + val blockId = optionalBlockId.getOrElse(nextBlockId) + val time = System.currentTimeMillis + blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], + storageLevel, tellMaster = true) + logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") + reportPushedBlock(blockId, arrayBuffer.size, optionalMetadata) + } + + /** Store a iterator of received data as a data block into Spark's memory. */ + def pushIterator( + iterator: Iterator[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + val blockId = optionalBlockId.getOrElse(nextBlockId) + val time = System.currentTimeMillis + blockManager.put(blockId, iterator, storageLevel, tellMaster = true) + logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") + reportPushedBlock(blockId, -1, optionalMetadata) + } + + /** Store the bytes of received data as a data block into Spark's memory. */ + def pushBytes( + bytes: ByteBuffer, + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + val blockId = optionalBlockId.getOrElse(nextBlockId) + val time = System.currentTimeMillis + blockManager.putBytes(blockId, bytes, storageLevel, tellMaster = true) + logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") + reportPushedBlock(blockId, -1, optionalMetadata) + } + + /** Report pushed block */ + def reportPushedBlock(blockId: StreamBlockId, numRecords: Long, optionalMetadata: Option[Any]) { + val blockInfo = ReceivedBlockInfo(streamId, blockId, numRecords, optionalMetadata.orNull) + trackerActor ! AddBlock(blockInfo) + logDebug("Reported block " + blockId) + } + + /** Report error to the receiver tracker */ + def reportError(message: String, error: Throwable) { + val errorString = Option(error).map(Throwables.getStackTraceAsString).getOrElse("") + trackerActor ! ReportError(streamId, message, errorString) + logWarning("Reported error " + message + " - " + error) + } + + override def onReceiverStart() { + blockGenerator.start() + super.onReceiverStart() + } + + override def onReceiverStop(message: String, error: Option[Throwable]) { + super.onReceiverStop(message, error) + blockGenerator.stop() + logInfo("Deregistering receiver " + streamId) + val errorString = error.map(Throwables.getStackTraceAsString).getOrElse("") + val future = trackerActor.ask( + DeregisterReceiver(streamId, message, errorString))(askTimeout) + Await.result(future, askTimeout) + logInfo("Stopped receiver " + streamId) + } + + override def stop(message: String, error: Option[Throwable]) { + super.stop(message, error) + env.actorSystem.stop(actor) + } + + /** Generate new block ID */ + private def nextBlockId = StreamBlockId(streamId, newBlockId.getAndIncrement) +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index e564eccba2df5..374848358e700 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -38,6 +38,7 @@ private[streaming] class JobGenerator(jobScheduler: JobScheduler) extends Logging { private val ssc = jobScheduler.ssc + private val conf = ssc.conf private val graph = ssc.graph val clock = { @@ -93,26 +94,31 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { if (processReceivedData) { logInfo("Stopping JobGenerator gracefully") val timeWhenStopStarted = System.currentTimeMillis() - val stopTimeout = 10 * ssc.graph.batchDuration.milliseconds + val stopTimeout = conf.getLong( + "spark.streaming.gracefulStopTimeout", + 10 * ssc.graph.batchDuration.milliseconds + ) val pollTime = 100 // To prevent graceful stop to get stuck permanently def hasTimedOut = { val timedOut = System.currentTimeMillis() - timeWhenStopStarted > stopTimeout - if (timedOut) logWarning("Timed out while stopping the job generator") + if (timedOut) { + logWarning("Timed out while stopping the job generator (timeout = " + stopTimeout + ")") + } timedOut } // Wait until all the received blocks in the network input tracker has // been consumed by network input DStreams, and jobs have been generated with them logInfo("Waiting for all received blocks to be consumed for job generation") - while(!hasTimedOut && jobScheduler.networkInputTracker.hasMoreReceivedBlockIds) { + while(!hasTimedOut && jobScheduler.receiverTracker.hasMoreReceivedBlockIds) { Thread.sleep(pollTime) } logInfo("Waited for all received blocks to be consumed for job generation") // Stop generating jobs - val stopTime = timer.stop(false) + val stopTime = timer.stop(interruptTimer = false) graph.stop() logInfo("Stopped generation timer") @@ -214,7 +220,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { SparkEnv.set(ssc.env) Try(graph.generateJobs(time)) match { case Success(jobs) => - val receivedBlockInfo = graph.getNetworkInputStreams.map { stream => + val receivedBlockInfo = graph.getReceiverInputStreams.map { stream => val streamId = stream.id val receivedBlockInfo = stream.getReceivedBlockInfo(time) (streamId, receivedBlockInfo) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index d9ada99b472ac..1b034b9fb187c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -46,7 +46,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { // These two are created only when scheduler starts. // eventActor not being null means the scheduler has been started and not stopped - var networkInputTracker: NetworkInputTracker = null + var receiverTracker: ReceiverTracker = null private var eventActor: ActorRef = null @@ -61,8 +61,8 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { }), "JobScheduler") listenerBus.start() - networkInputTracker = new NetworkInputTracker(ssc) - networkInputTracker.start() + receiverTracker = new ReceiverTracker(ssc) + receiverTracker.start() jobGenerator.start() logInfo("Started JobScheduler") } @@ -72,7 +72,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { logDebug("Stopping JobScheduler") // First, stop receiving - networkInputTracker.stop() + receiverTracker.stop() // Second, stop generating jobs. If it has to process all received data, // then this will wait for all the processing through JobScheduler to be over. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala similarity index 68% rename from streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala rename to streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 438e72a7ced89..3d2537f6f23dc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -21,12 +21,11 @@ import scala.collection.mutable.{HashMap, SynchronizedMap, SynchronizedQueue} import scala.language.existentials import akka.actor._ - import org.apache.spark.{Logging, SparkEnv, SparkException} import org.apache.spark.SparkContext._ import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.{StreamingContext, Time} -import org.apache.spark.streaming.dstream.{NetworkReceiver, StopReceiver} +import org.apache.spark.streaming.receiver.{Receiver, ReceiverSupervisorImpl, StopReceiver} import org.apache.spark.util.AkkaUtils /** Information about receiver */ @@ -34,7 +33,7 @@ case class ReceiverInfo(streamId: Int, typ: String, location: String) { override def toString = s"$typ-$streamId" } -/** Information about blocks received by the network receiver */ +/** Information about blocks received by the receiver */ case class ReceivedBlockInfo( streamId: Int, blockId: StreamBlockId, @@ -43,20 +42,21 @@ case class ReceivedBlockInfo( ) /** - * Messages used by the NetworkReceiver and the NetworkInputTracker to communicate + * Messages used by the NetworkReceiver and the ReceiverTracker to communicate * with each other. */ -private[streaming] sealed trait NetworkInputTrackerMessage +private[streaming] sealed trait ReceiverTrackerMessage private[streaming] case class RegisterReceiver( streamId: Int, typ: String, host: String, receiverActor: ActorRef - ) extends NetworkInputTrackerMessage + ) extends ReceiverTrackerMessage private[streaming] case class AddBlock(receivedBlockInfo: ReceivedBlockInfo) - extends NetworkInputTrackerMessage -private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) - extends NetworkInputTrackerMessage + extends ReceiverTrackerMessage +private[streaming] case class ReportError(streamId: Int, message: String, error: String) +private[streaming] case class DeregisterReceiver(streamId: Int, msg: String, error: String) + extends ReceiverTrackerMessage /** * This class manages the execution of the receivers of NetworkInputDStreams. Instance of @@ -64,11 +64,11 @@ private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) * has been called because it needs the final set of input streams at the time of instantiation. */ private[streaming] -class NetworkInputTracker(ssc: StreamingContext) extends Logging { +class ReceiverTracker(ssc: StreamingContext) extends Logging { - val networkInputStreams = ssc.graph.getNetworkInputStreams() - val networkInputStreamMap = Map(networkInputStreams.map(x => (x.id, x)): _*) - val receiverExecutor = new ReceiverExecutor() + val receiverInputStreams = ssc.graph.getReceiverInputStreams() + val receiverInputStreamMap = Map(receiverInputStreams.map(x => (x.id, x)): _*) + val receiverExecutor = new ReceiverLauncher() val receiverInfo = new HashMap[Int, ActorRef] with SynchronizedMap[Int, ActorRef] val receivedBlockInfo = new HashMap[Int, SynchronizedQueue[ReceivedBlockInfo]] with SynchronizedMap[Int, SynchronizedQueue[ReceivedBlockInfo]] @@ -83,27 +83,27 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { /** Start the actor and receiver execution thread. */ def start() = synchronized { if (actor != null) { - throw new SparkException("NetworkInputTracker already started") + throw new SparkException("ReceiverTracker already started") } - if (!networkInputStreams.isEmpty) { - actor = ssc.env.actorSystem.actorOf(Props(new NetworkInputTrackerActor), - "NetworkInputTracker") + if (!receiverInputStreams.isEmpty) { + actor = ssc.env.actorSystem.actorOf(Props(new ReceiverTrackerActor), + "ReceiverTracker") receiverExecutor.start() - logInfo("NetworkInputTracker started") + logInfo("ReceiverTracker started") } } /** Stop the receiver execution thread. */ def stop() = synchronized { - if (!networkInputStreams.isEmpty && actor != null) { + if (!receiverInputStreams.isEmpty && actor != null) { // First, stop the receivers receiverExecutor.stop() // Finally, stop the actor ssc.env.actorSystem.stop(actor) actor = null - logInfo("NetworkInputTracker stopped") + logInfo("ReceiverTracker stopped") } } @@ -126,20 +126,26 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { receiverActor: ActorRef, sender: ActorRef ) { - if (!networkInputStreamMap.contains(streamId)) { + if (!receiverInputStreamMap.contains(streamId)) { throw new Exception("Register received for unexpected id " + streamId) } receiverInfo += ((streamId, receiverActor)) ssc.scheduler.listenerBus.post(StreamingListenerReceiverStarted( ReceiverInfo(streamId, typ, host) )) - logInfo("Registered receiver for network stream " + streamId + " from " + sender.path.address) + logInfo("Registered receiver for stream " + streamId + " from " + sender.path.address) } /** Deregister a receiver */ - def deregisterReceiver(streamId: Int, message: String) { + def deregisterReceiver(streamId: Int, message: String, error: String) { receiverInfo -= streamId - logError("Deregistered receiver for network stream " + streamId + " with message:\n" + message) + ssc.scheduler.listenerBus.post(StreamingListenerReceiverStopped(streamId, message, error)) + val messageWithError = if (error != null && !error.isEmpty) { + s"$message - $error" + } else { + s"$message" + } + logError(s"Deregistered receiver for stream $streamId: $messageWithError") } /** Add new blocks for the given stream */ @@ -149,27 +155,40 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { receivedBlockInfo.blockId) } + /** Report error sent by a receiver */ + def reportError(streamId: Int, message: String, error: String) { + ssc.scheduler.listenerBus.post(StreamingListenerReceiverError(streamId, message, error)) + val messageWithError = if (error != null && !error.isEmpty) { + s"$message - $error" + } else { + s"$message" + } + logWarning(s"Error reported by receiver for stream $streamId: $messageWithError") + } + /** Check if any blocks are left to be processed */ def hasMoreReceivedBlockIds: Boolean = { !receivedBlockInfo.values.forall(_.isEmpty) } /** Actor to receive messages from the receivers. */ - private class NetworkInputTrackerActor extends Actor { + private class ReceiverTrackerActor extends Actor { def receive = { case RegisterReceiver(streamId, typ, host, receiverActor) => registerReceiver(streamId, typ, host, receiverActor, sender) sender ! true case AddBlock(receivedBlockInfo) => addBlocks(receivedBlockInfo) - case DeregisterReceiver(streamId, message) => - deregisterReceiver(streamId, message) + case ReportError(streamId, message, error) => + reportError(streamId, message, error) + case DeregisterReceiver(streamId, message, error) => + deregisterReceiver(streamId, message, error) sender ! true } } /** This thread class runs all the receivers on the cluster. */ - class ReceiverExecutor { + class ReceiverLauncher { @transient val env = ssc.env @transient val thread = new Thread() { override def run() { @@ -177,7 +196,7 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { SparkEnv.set(env) startReceivers() } catch { - case ie: InterruptedException => logInfo("ReceiverExecutor interrupted") + case ie: InterruptedException => logInfo("ReceiverLauncher interrupted") } } } @@ -203,37 +222,39 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { } /** - * Get the receivers from the NetworkInputDStreams, distributes them to the + * Get the receivers from the ReceiverInputDStreams, distributes them to the * worker nodes as a parallel collection, and runs them. */ private def startReceivers() { - val receivers = networkInputStreams.map(nis => { + val receivers = receiverInputStreams.map(nis => { val rcvr = nis.getReceiver() - rcvr.setStreamId(nis.id) + rcvr.setReceiverId(nis.id) rcvr }) // Right now, we only honor preferences if all receivers have them - val hasLocationPreferences = receivers.map(_.getLocationPreference().isDefined) - .reduce(_ && _) + val hasLocationPreferences = receivers.map(_.preferredLocation.isDefined).reduce(_ && _) // Create the parallel collection of receivers to distributed them on the worker nodes val tempRDD = if (hasLocationPreferences) { - val receiversWithPreferences = - receivers.map(r => (r, Seq(r.getLocationPreference().toString))) - ssc.sc.makeRDD[NetworkReceiver[_]](receiversWithPreferences) + val receiversWithPreferences = receivers.map(r => (r, Seq(r.preferredLocation.get))) + ssc.sc.makeRDD[Receiver[_]](receiversWithPreferences) } else { ssc.sc.makeRDD(receivers, receivers.size) } // Function to start the receiver on the worker node - val startReceiver = (iterator: Iterator[NetworkReceiver[_]]) => { + val startReceiver = (iterator: Iterator[Receiver[_]]) => { if (!iterator.hasNext) { - throw new Exception("Could not start receiver as details not found.") + throw new SparkException( + "Could not start receiver as object not found.") } - iterator.next().start() + val receiver = iterator.next() + val executor = new ReceiverSupervisorImpl(receiver, SparkEnv.get) + executor.start() + executor.awaitTermination() } // Run the dummy Spark job to ensure that all slaves have registered. // This avoids all the receivers to be scheduled on the same node. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala index 5db40ebbeb1de..9d6ec1fa33549 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming.scheduler import scala.collection.mutable.Queue + import org.apache.spark.util.Distribution /** Base trait for events related to StreamingListener */ @@ -26,8 +27,13 @@ sealed trait StreamingListenerEvent case class StreamingListenerBatchSubmitted(batchInfo: BatchInfo) extends StreamingListenerEvent case class StreamingListenerBatchCompleted(batchInfo: BatchInfo) extends StreamingListenerEvent case class StreamingListenerBatchStarted(batchInfo: BatchInfo) extends StreamingListenerEvent + case class StreamingListenerReceiverStarted(receiverInfo: ReceiverInfo) extends StreamingListenerEvent +case class StreamingListenerReceiverError(streamId: Int, message: String, error: String) + extends StreamingListenerEvent +case class StreamingListenerReceiverStopped(streamId: Int, message: String, error: String) + extends StreamingListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ private[scheduler] case object StreamingListenerShutdown extends StreamingListenerEvent @@ -41,14 +47,20 @@ trait StreamingListener { /** Called when a receiver has been started */ def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { } + /** Called when a receiver has reported an error */ + def onReceiverError(receiverError: StreamingListenerReceiverError) { } + + /** Called when a receiver has been stopped */ + def onReceiverStopped(receiverStopped: StreamingListenerReceiverStopped) { } + /** Called when a batch of jobs has been submitted for processing. */ def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) { } - /** Called when processing of a batch of jobs has completed. */ - def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { } - /** Called when processing of a batch of jobs has started. */ def onBatchStarted(batchStarted: StreamingListenerBatchStarted) { } + + /** Called when processing of a batch of jobs has completed. */ + def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala index ea03dfc7bfeea..398724d9e8130 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala @@ -40,6 +40,10 @@ private[spark] class StreamingListenerBus() extends Logging { event match { case receiverStarted: StreamingListenerReceiverStarted => listeners.foreach(_.onReceiverStarted(receiverStarted)) + case receiverError: StreamingListenerReceiverError => + listeners.foreach(_.onReceiverError(receiverError)) + case receiverStopped: StreamingListenerReceiverStopped => + listeners.foreach(_.onReceiverStopped(receiverStopped)) case batchSubmitted: StreamingListenerBatchSubmitted => listeners.foreach(_.onBatchSubmitted(batchSubmitted)) case batchStarted: StreamingListenerBatchStarted => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index 8b025b09ed34d..bf637c1446314 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -62,8 +62,8 @@ private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends St totalCompletedBatches += 1L } - def numNetworkReceivers = synchronized { - ssc.graph.getNetworkInputStreams().size + def numReceivers = synchronized { + ssc.graph.getReceiverInputStreams().size } def numTotalCompletedBatches: Long = synchronized { @@ -101,7 +101,7 @@ private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends St def receivedRecordsDistributions: Map[Int, Option[Distribution]] = synchronized { val latestBatchInfos = retainedBatches.reverse.take(batchInfoLimit) val latestBlockInfos = latestBatchInfos.map(_.receivedBlockInfo) - (0 until numNetworkReceivers).map { receiverId => + (0 until numReceivers).map { receiverId => val blockInfoOfParticularReceiver = latestBlockInfos.map { batchInfo => batchInfo.get(receiverId).getOrElse(Array.empty) } @@ -117,11 +117,11 @@ private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends St def lastReceivedBatchRecords: Map[Int, Long] = { val lastReceivedBlockInfoOption = lastReceivedBatch.map(_.receivedBlockInfo) lastReceivedBlockInfoOption.map { lastReceivedBlockInfo => - (0 until numNetworkReceivers).map { receiverId => + (0 until numReceivers).map { receiverId => (receiverId, lastReceivedBlockInfo(receiverId).map(_.numRecords).sum) }.toMap }.getOrElse { - (0 until numNetworkReceivers).map(receiverId => (receiverId, 0L)).toMap + (0 until numReceivers).map(receiverId => (receiverId, 0L)).toMap } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 6607437db560a..8fe1219356cdc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -40,7 +40,7 @@ private[ui] class StreamingPage(parent: StreamingTab) val content = generateBasicStats() ++

      ++

      Statistics over last {listener.retainedCompletedBatches.size} processed batches

      ++ - generateNetworkStatsTable() ++ + generateReceiverStats() ++ generateBatchStatsTable() UIUtils.headerSparkPage( content, parent.basePath, parent.appName, "Streaming", parent.headerTabs, parent, Some(5000)) @@ -57,7 +57,7 @@ private[ui] class StreamingPage(parent: StreamingTab) Time since start: {formatDurationVerbose(timeSinceStart)}
    • - Network receivers: {listener.numNetworkReceivers} + Network receivers: {listener.numReceivers}
    • Batch interval: {formatDurationVerbose(listener.batchDuration)} @@ -71,8 +71,8 @@ private[ui] class StreamingPage(parent: StreamingTab) } - /** Generate stats of data received over the network the streaming program */ - private def generateNetworkStatsTable(): Seq[Node] = { + /** Generate stats of data received by the receivers in the streaming program */ + private def generateReceiverStats(): Seq[Node] = { val receivedRecordDistributions = listener.receivedRecordsDistributions val lastBatchReceivedRecord = listener.lastReceivedBatchRecords val table = if (receivedRecordDistributions.size > 0) { @@ -86,13 +86,13 @@ private[ui] class StreamingPage(parent: StreamingTab) "75th percentile rate\n[records/sec]", "Maximum rate\n[records/sec]" ) - val dataRows = (0 until listener.numNetworkReceivers).map { receiverId => + val dataRows = (0 until listener.numReceivers).map { receiverId => val receiverInfo = listener.receiverInfo(receiverId) val receiverName = receiverInfo.map(_.toString).getOrElse(s"Receiver-$receiverId") val receiverLocation = receiverInfo.map(_.location).getOrElse(emptyCell) - val receiverLastBatchRecords = formatDurationVerbose(lastBatchReceivedRecord(receiverId)) + val receiverLastBatchRecords = formatNumber(lastBatchReceivedRecord(receiverId)) val receivedRecordStats = receivedRecordDistributions(receiverId).map { d => - d.getQuantiles().map(r => formatDurationVerbose(r.toLong)) + d.getQuantiles().map(r => formatNumber(r.toLong)) }.getOrElse { Seq(emptyCell, emptyCell, emptyCell, emptyCell, emptyCell) } @@ -104,8 +104,8 @@ private[ui] class StreamingPage(parent: StreamingTab) } val content = -
      Network Input Statistics
      ++ -
      {table.getOrElse("No network receivers")}
      +
      Receiver Statistics
      ++ +
      {table.getOrElse("No receivers")}
      content } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala index e016377c94c0d..1a616a0434f2c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala @@ -77,7 +77,9 @@ class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name: def stop(interruptTimer: Boolean): Long = synchronized { if (!stopped) { stopped = true - if (interruptTimer) thread.interrupt() + if (interruptTimer) { + thread.interrupt() + } thread.join() logInfo("Stopped timer for " + name + " after time " + prevTime) } diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index a0b1bbc34fa7c..f9bfb9b7444cc 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.streaming.api.java.*; import scala.Tuple2; import org.junit.Assert; @@ -36,10 +37,6 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.*; import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.api.java.JavaDStream; -import org.apache.spark.streaming.api.java.JavaDStreamLike; -import org.apache.spark.streaming.api.java.JavaPairDStream; -import org.apache.spark.streaming.api.java.JavaStreamingContext; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; @@ -1668,7 +1665,7 @@ public Integer call(String s) throws Exception { // InputStream functionality is deferred to the existing Scala tests. @Test public void testSocketTextStream() { - JavaDStream test = ssc.socketTextStream("localhost", 12345); + JavaReceiverInputDStream test = ssc.socketTextStream("localhost", 12345); } @Test @@ -1701,6 +1698,6 @@ public void testTextFileStream() { @Test public void testRawSocketStream() { - JavaDStream test = ssc.rawSocketStream("localhost", 12345); + JavaReceiverInputDStream test = ssc.rawSocketStream("localhost", 12345); } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 952511d411a8e..46b7f63b6567e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -36,10 +36,9 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.dstream.NetworkReceiver -import org.apache.spark.streaming.receivers.Receiver import org.apache.spark.streaming.util.ManualClock import org.apache.spark.util.Utils +import org.apache.spark.streaming.receiver.{ActorHelper, Receiver} class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { @@ -207,7 +206,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // set up the network stream using the test receiver val ssc = new StreamingContext(conf, batchDuration) - val networkStream = ssc.networkStream[Int](testReceiver) + val networkStream = ssc.receiverStream[Int](testReceiver) val countStream = networkStream.count val outputBuffer = new ArrayBuffer[Seq[Long]] with SynchronizedBuffer[Seq[Long]] val outputStream = new TestOutputStream(countStream, outputBuffer) @@ -301,7 +300,7 @@ object TestServer { } /** This is an actor for testing actor input stream */ -class TestActor(port: Int) extends Actor with Receiver { +class TestActor(port: Int) extends Actor with ActorHelper { def bytesToString(byteString: ByteString) = byteString.utf8String @@ -309,24 +308,22 @@ class TestActor(port: Int) extends Actor with Receiver { def receive = { case IO.Read(socket, bytes) => - pushBlock(bytesToString(bytes)) + store(bytesToString(bytes)) } } /** This is a receiver to test multiple threads inserting data using block generator */ class MultiThreadTestReceiver(numThreads: Int, numRecordsPerThread: Int) - extends NetworkReceiver[Int] { + extends Receiver[Int](StorageLevel.MEMORY_ONLY_SER) with Logging { lazy val executorPool = Executors.newFixedThreadPool(numThreads) - lazy val blockGenerator = new BlockGenerator(StorageLevel.MEMORY_ONLY) lazy val finishCount = new AtomicInteger(0) - protected def onStart() { - blockGenerator.start() + def onStart() { (1 to numThreads).map(threadId => { val runnable = new Runnable { def run() { (1 to numRecordsPerThread).foreach(i => - blockGenerator += (threadId * numRecordsPerThread + i) ) + store(threadId * numRecordsPerThread + i) ) if (finishCount.incrementAndGet == numThreads) { MultiThreadTestReceiver.haveAllThreadsFinished = true } @@ -337,7 +334,7 @@ class MultiThreadTestReceiver(numThreads: Int, numRecordsPerThread: Int) }) } - protected def onStop() { + def onStop() { executorPool.shutdown() } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala new file mode 100644 index 0000000000000..5c0415ad14ebb --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala @@ -0,0 +1,249 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.SparkConf +import org.apache.spark.storage.{StorageLevel, StreamBlockId} +import org.apache.spark.streaming.receiver.{BlockGenerator, BlockGeneratorListener, Receiver, ReceiverSupervisor} +import org.scalatest.FunSuite +import org.scalatest.concurrent.Timeouts +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ + +/** Testsuite for testing the network receiver behavior */ +class NetworkReceiverSuite extends FunSuite with Timeouts { + + test("network receiver life cycle") { + + val receiver = new FakeReceiver + val executor = new FakeReceiverSupervisor(receiver) + + assert(executor.isAllEmpty) + + // Thread that runs the executor + val executingThread = new Thread() { + override def run() { + executor.start() + executor.awaitTermination() + } + } + + // Start the receiver + executingThread.start() + + // Verify that the receiver + intercept[Exception] { + failAfter(200 millis) { + executingThread.join() + } + } + + // Verify that receiver was started + assert(receiver.onStartCalled) + assert(executor.isReceiverStarted) + assert(receiver.isStarted) + assert(!receiver.isStopped()) + assert(receiver.otherThread.isAlive) + eventually(timeout(100 millis), interval(10 millis)) { + assert(receiver.receiving) + } + + // Verify whether the data stored by the receiver was sent to the executor + val byteBuffer = ByteBuffer.allocate(100) + val arrayBuffer = new ArrayBuffer[Int]() + val iterator = arrayBuffer.iterator + receiver.store(1) + receiver.store(byteBuffer) + receiver.store(arrayBuffer) + receiver.store(iterator) + assert(executor.singles.size === 1) + assert(executor.singles.head === 1) + assert(executor.byteBuffers.size === 1) + assert(executor.byteBuffers.head.eq(byteBuffer)) + assert(executor.iterators.size === 1) + assert(executor.iterators.head.eq(iterator)) + assert(executor.arrayBuffers.size === 1) + assert(executor.arrayBuffers.head.eq(arrayBuffer)) + + // Verify whether the exceptions reported by the receiver was sent to the executor + val exception = new Exception + receiver.reportError("Error", exception) + assert(executor.errors.size === 1) + assert(executor.errors.head.eq(exception)) + + // Verify restarting actually stops and starts the receiver + receiver.restart("restarting", null, 100) + assert(receiver.isStopped) + assert(receiver.onStopCalled) + eventually(timeout(1000 millis), interval(100 millis)) { + assert(receiver.onStartCalled) + assert(executor.isReceiverStarted) + assert(receiver.isStarted) + assert(!receiver.isStopped) + assert(receiver.receiving) + } + + // Verify that stopping actually stops the thread + failAfter(100 millis) { + receiver.stop("test") + assert(receiver.isStopped) + assert(!receiver.otherThread.isAlive) + + // The thread that started the executor should complete + // as stop() stops everything + executingThread.join() + } + } + + test("block generator") { + val blockGeneratorListener = new FakeBlockGeneratorListener + val blockInterval = 200 + val conf = new SparkConf().set("spark.streaming.blockInterval", blockInterval.toString) + val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) + val expectedBlocks = 5 + val waitTime = expectedBlocks * blockInterval + (blockInterval / 2) + val generatedData = new ArrayBuffer[Int] + + // Generate blocks + val startTime = System.currentTimeMillis() + blockGenerator.start() + var count = 0 + while(System.currentTimeMillis - startTime < waitTime) { + blockGenerator += count + generatedData += count + count += 1 + Thread.sleep(10) + } + blockGenerator.stop() + + val recordedData = blockGeneratorListener.arrayBuffers.flatten + assert(blockGeneratorListener.arrayBuffers.size > 0) + assert(recordedData.toSet === generatedData.toSet) + } + + /** + * An implementation of NetworkReceiver that is used for testing a receiver's life cycle. + */ + class FakeReceiver extends Receiver[Int](StorageLevel.MEMORY_ONLY) { + var otherThread: Thread = null + var receiving = false + var onStartCalled = false + var onStopCalled = false + + def onStart() { + otherThread = new Thread() { + override def run() { + receiving = true + while(!isStopped()) { + Thread.sleep(10) + } + } + } + onStartCalled = true + otherThread.start() + + } + + def onStop() { + onStopCalled = true + otherThread.join() + } + + def reset() { + receiving = false + onStartCalled = false + onStopCalled = false + } + } + + /** + * An implementation of NetworkReceiverExecutor used for testing a NetworkReceiver. + * Instead of storing the data in the BlockManager, it stores all the data in a local buffer + * that can used for verifying that the data has been forwarded correctly. + */ + class FakeReceiverSupervisor(receiver: FakeReceiver) + extends ReceiverSupervisor(receiver, new SparkConf()) { + val singles = new ArrayBuffer[Any] + val byteBuffers = new ArrayBuffer[ByteBuffer] + val iterators = new ArrayBuffer[Iterator[_]] + val arrayBuffers = new ArrayBuffer[ArrayBuffer[_]] + val errors = new ArrayBuffer[Throwable] + + /** Check if all data structures are clean */ + def isAllEmpty = { + singles.isEmpty && byteBuffers.isEmpty && iterators.isEmpty && + arrayBuffers.isEmpty && errors.isEmpty + } + + def pushSingle(data: Any) { + singles += data + } + + def pushBytes( + bytes: ByteBuffer, + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + byteBuffers += bytes + } + + def pushIterator( + iterator: Iterator[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + iterators += iterator + } + + def pushArrayBuffer( + arrayBuffer: ArrayBuffer[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + arrayBuffers += arrayBuffer + } + + def reportError(message: String, throwable: Throwable) { + errors += throwable + } + } + + /** + * An implementation of BlockGeneratorListener that is used to test the BlockGenerator. + */ + class FakeBlockGeneratorListener(pushDelay: Long = 0) extends BlockGeneratorListener { + // buffer of data received as ArrayBuffers + val arrayBuffers = new ArrayBuffer[ArrayBuffer[Int]] + val errors = new ArrayBuffer[Throwable] + + def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) { + val bufferOfInts = arrayBuffer.map(_.asInstanceOf[Int]) + arrayBuffers += bufferOfInts + Thread.sleep(0) + } + + def onError(message: String, throwable: Throwable) { + errors += throwable + } + } +} + diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index ad5367ab941a2..6d14b1f785db3 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -21,7 +21,8 @@ import java.util.concurrent.atomic.AtomicInteger import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException} import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.dstream.{DStream, NetworkReceiver} +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.receiver.Receiver import org.apache.spark.util.{MetadataCleaner, Utils} import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.concurrent.Timeouts @@ -181,15 +182,15 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w conf.set("spark.cleaner.ttl", "3600") sc = new SparkContext(conf) for (i <- 1 to 4) { - logInfo("==================================") - ssc = new StreamingContext(sc, batchDuration) + logInfo("==================================\n\n\n") + ssc = new StreamingContext(sc, Milliseconds(100)) var runningCount = 0 TestReceiver.counter.set(1) val input = ssc.networkStream(new TestReceiver) input.count.foreachRDD(rdd => { val count = rdd.first() - logInfo("Count = " + count) runningCount += count.toInt + logInfo("Count = " + count + ", Running count = " + runningCount) }) ssc.start() ssc.awaitTermination(500) @@ -216,12 +217,12 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc.start() } - // test whether waitForStop() exits after give amount of time + // test whether awaitTermination() exits after give amount of time failAfter(1000 millis) { ssc.awaitTermination(500) } - // test whether waitForStop() does not exit if not time is given + // test whether awaitTermination() does not exit if not time is given val exception = intercept[Exception] { failAfter(1000 millis) { ssc.awaitTermination() @@ -276,23 +277,26 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w class TestException(msg: String) extends Exception(msg) /** Custom receiver for testing whether all data received by a receiver gets processed or not */ -class TestReceiver extends NetworkReceiver[Int] { - protected lazy val blockGenerator = new BlockGenerator(StorageLevel.MEMORY_ONLY) - protected def onStart() { - blockGenerator.start() - logInfo("BlockGenerator started on thread " + receivingThread) - try { - while(true) { - blockGenerator += TestReceiver.counter.getAndIncrement - Thread.sleep(0) +class TestReceiver extends Receiver[Int](StorageLevel.MEMORY_ONLY) with Logging { + + var receivingThreadOption: Option[Thread] = None + + def onStart() { + val thread = new Thread() { + override def run() { + logInfo("Receiving started") + while (!isStopped) { + store(TestReceiver.counter.getAndIncrement) + } + logInfo("Receiving stopped at count value of " + TestReceiver.counter.get()) } - } finally { - logInfo("Receiving stopped at count value of " + TestReceiver.counter.get()) } + receivingThreadOption = Some(thread) + thread.start() } - protected def onStop() { - blockGenerator.stop() + def onStop() { + // no cleanup to be done, the receiving thread should stop on it own } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 9e0f2c900e8ba..542c697ae3127 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -17,10 +17,19 @@ package org.apache.spark.streaming -import org.apache.spark.streaming.scheduler._ import scala.collection.mutable.ArrayBuffer -import org.scalatest.matchers.ShouldMatchers +import scala.concurrent.Future +import scala.concurrent.ExecutionContext.Implicits.global + +import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.streaming.scheduler._ + +import org.scalatest.matchers.ShouldMatchers +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ +import org.apache.spark.Logging class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers { @@ -32,7 +41,7 @@ class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers { override def batchDuration = Milliseconds(100) override def actuallyWait = true - test("basic BatchInfo generation") { + test("batch info reporting") { val ssc = setupStreams(input, operation) val collector = new BatchInfoCollector ssc.addStreamingListener(collector) @@ -54,6 +63,31 @@ class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers { isInIncreasingOrder(batchInfos.map(_.processingEndTime.get)) should be (true) } + test("receiver info reporting") { + val ssc = new StreamingContext("local[2]", "test", Milliseconds(1000)) + val inputStream = ssc.networkStream(new StreamingListenerSuiteReceiver) + inputStream.foreachRDD(_.count) + + val collector = new ReceiverInfoCollector + ssc.addStreamingListener(collector) + + ssc.start() + try { + eventually(timeout(1000 millis), interval(20 millis)) { + collector.startedReceiverInfo should have size 1 + collector.startedReceiverInfo(0).streamId should equal (0) + collector.stoppedReceiverStreamIds should have size 1 + collector.stoppedReceiverStreamIds(0) should equal (0) + collector.receiverErrors should have size 1 + collector.receiverErrors(0)._1 should equal (0) + collector.receiverErrors(0)._2 should include ("report error") + collector.receiverErrors(0)._3 should include ("report exception") + } + } finally { + ssc.stop() + } + } + /** Check if a sequence of numbers is in increasing order */ def isInIncreasingOrder(seq: Seq[Long]): Boolean = { for(i <- 1 until seq.size) { @@ -61,12 +95,46 @@ class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers { } true } +} + +/** Listener that collects information on processed batches */ +class BatchInfoCollector extends StreamingListener { + val batchInfos = new ArrayBuffer[BatchInfo] + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { + batchInfos += batchCompleted.batchInfo + } +} + +/** Listener that collects information on processed batches */ +class ReceiverInfoCollector extends StreamingListener { + val startedReceiverInfo = new ArrayBuffer[ReceiverInfo] + val stoppedReceiverStreamIds = new ArrayBuffer[Int]() + val receiverErrors = new ArrayBuffer[(Int, String, String)]() + + override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { + startedReceiverInfo += receiverStarted.receiverInfo + } + + override def onReceiverStopped(receiverStopped: StreamingListenerReceiverStopped) { + stoppedReceiverStreamIds += receiverStopped.streamId + } + + override def onReceiverError(receiverError: StreamingListenerReceiverError) { + receiverErrors += ((receiverError.streamId, receiverError.message, receiverError.error)) + } +} - /** Listener that collects information on processed batches */ - class BatchInfoCollector extends StreamingListener { - val batchInfos = new ArrayBuffer[BatchInfo] - override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { - batchInfos += batchCompleted.batchInfo +class StreamingListenerSuiteReceiver extends Receiver[Any](StorageLevel.MEMORY_ONLY) with Logging { + def onStart() { + Future { + logInfo("Started receiver and sleeping") + Thread.sleep(10) + logInfo("Reporting error and sleeping") + reportError("test report error", new Exception("test report exception")) + Thread.sleep(10) + logInfo("Stopping") + stop("test stop error") } } + def onStop() { } } From fc7838470465474f777bd17791c1bb5f9c348521 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 21 Apr 2014 21:57:40 -0700 Subject: [PATCH 122/641] [SPARK-1439, SPARK-1440] Generate unified Scaladoc across projects and Javadocs I used the sbt-unidoc plugin (https://github.com/sbt/sbt-unidoc) to create a unified Scaladoc of our public packages, and generate Javadocs as well. One limitation is that I haven't found an easy way to exclude packages in the Javadoc; there is a SBT task that identifies Java sources to run javadoc on, but it's been very difficult to modify it from outside to change what is set in the unidoc package. Some SBT-savvy people should help with this. The Javadoc site also lacks package-level descriptions and things like that, so we may want to look into that. We may decide not to post these right now if it's too limited compared to the Scala one. Example of the built doc site: http://people.csail.mit.edu/matei/spark-unified-docs/ Author: Matei Zaharia This patch had conflicts when merged, resolved by Committer: Patrick Wendell Closes #457 from mateiz/better-docs and squashes the following commits: a63d4a3 [Matei Zaharia] Skip Java/Scala API docs for Python package 5ea1f43 [Matei Zaharia] Fix links to Java classes in Java guide, fix some JS for scrolling to anchors on page load f05abc0 [Matei Zaharia] Don't include java.lang package names 995e992 [Matei Zaharia] Skip internal packages and class names with $ in JavaDoc a14a93c [Matei Zaharia] typo 76ce64d [Matei Zaharia] Add groups to Javadoc index page, and a first package-info.java ed6f994 [Matei Zaharia] Generate JavaDoc as well, add titles, update doc site to use unified docs acb993d [Matei Zaharia] Add Unidoc plugin for the projects we want Unidoced --- .../java/org/apache/spark/package-info.java | 23 ++++ .../main/scala/org/apache/spark/package.scala | 2 +- docs/_layouts/global.html | 105 +++--------------- docs/_plugins/copy_api_dirs.rb | 65 +++++------ docs/api.md | 13 +-- docs/configuration.md | 8 +- docs/graphx-programming-guide.md | 62 +++++------ docs/index.md | 10 +- docs/java-programming-guide.md | 55 ++++----- docs/js/main.js | 18 +++ docs/mllib-classification-regression.md | 14 +-- docs/mllib-clustering.md | 2 +- docs/mllib-collaborative-filtering.md | 2 +- docs/mllib-guide.md | 10 +- docs/mllib-optimization.md | 8 +- docs/python-programming-guide.md | 4 +- docs/quick-start.md | 6 +- docs/scala-programming-guide.md | 10 +- docs/sql-programming-guide.md | 20 ++-- docs/streaming-custom-receivers.md | 4 +- docs/streaming-programming-guide.md | 56 +++++----- docs/tuning.md | 4 +- project/SparkBuild.scala | 74 ++++++++++-- project/plugins.sbt | 1 + python/epydoc.conf | 4 +- 25 files changed, 296 insertions(+), 284 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/package-info.java diff --git a/core/src/main/java/org/apache/spark/package-info.java b/core/src/main/java/org/apache/spark/package-info.java new file mode 100644 index 0000000000000..4426c7afcebdd --- /dev/null +++ b/core/src/main/java/org/apache/spark/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Core Spark classes in Scala. A few classes here, such as {@link org.apache.spark.Accumulator} + * and {@link org.apache.spark.storage.StorageLevel}, are also used in Java, but the + * {@link org.apache.spark.api.java} package contains the main Java API. + */ +package org.apache.spark; diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 59bbb1171f239..5cdbc306e56a0 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -30,7 +30,7 @@ package org.apache * type (e.g. RDD[(Int, Int)] through implicit conversions when you * `import org.apache.spark.SparkContext._`. * - * Java programmers should reference the [[spark.api.java]] package + * Java programmers should reference the [[org.apache.spark.api.java]] package * for Spark programming APIs in Java. * * Classes and methods marked with diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 5d4dbb7a9c3a9..8b543de574622 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -76,32 +76,9 @@
    • @@ -140,33 +117,6 @@

      {{ page.title }}

      {{ content }} - - - - - @@ -174,42 +124,23 @@

      Heading

      - + + + - - - - - diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 05f0bd47a88a5..2dbbbf6feb4b8 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -20,47 +20,48 @@ if not (ENV['SKIP_API'] == '1' or ENV['SKIP_SCALADOC'] == '1') # Build Scaladoc for Java/Scala - core_projects = ["core", "examples", "repl", "bagel", "graphx", "streaming", "mllib"] - external_projects = ["flume", "kafka", "mqtt", "twitter", "zeromq"] - sql_projects = ["catalyst", "core", "hive"] - projects = core_projects - projects = projects + external_projects.map { |project_name| "external/" + project_name } - projects = projects + sql_projects.map { |project_name| "sql/" + project_name } - - puts "Moving to project root and building scaladoc." + puts "Moving to project root and building API docs." curr_dir = pwd cd("..") - puts "Running 'sbt/sbt doc hive/doc' from " + pwd + "; this may take a few minutes..." - puts `sbt/sbt doc hive/doc` + puts "Running 'sbt/sbt compile unidoc' from " + pwd + "; this may take a few minutes..." + puts `sbt/sbt compile unidoc` puts "Moving back into docs dir." cd("docs") - # Copy over the scaladoc from each project into the docs directory. + # Copy over the unified ScalaDoc for all projects to api/scala. # This directory will be copied over to _site when `jekyll` command is run. - projects.each do |project_name| - source = "../" + project_name + "/target/scala-2.10/api" - dest = "api/" + project_name + source = "../target/scala-2.10/unidoc" + dest = "api/scala" + + puts "Making directory " + dest + mkdir_p dest + + # From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't. + puts "cp -r " + source + "/. " + dest + cp_r(source + "/.", dest) + + # Append custom JavaScript + js = File.readlines("./js/api-docs.js") + js_file = dest + "/lib/template.js" + File.open(js_file, 'a') { |f| f.write("\n" + js.join()) } - puts "making directory " + dest - mkdir_p dest + # Append custom CSS + css = File.readlines("./css/api-docs.css") + css_file = dest + "/lib/template.css" + File.open(css_file, 'a') { |f| f.write("\n" + css.join()) } - # From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't. - puts "cp -r " + source + "/. " + dest - cp_r(source + "/.", dest) + # Copy over the unified JavaDoc for all projects to api/java. + source = "../target/javaunidoc" + dest = "api/java" - # Append custom JavaScript - js = File.readlines("./js/api-docs.js") - js_file = dest + "/lib/template.js" - File.open(js_file, 'a') { |f| f.write("\n" + js.join()) } + puts "Making directory " + dest + mkdir_p dest - # Append custom CSS - css = File.readlines("./css/api-docs.css") - css_file = dest + "/lib/template.css" - File.open(css_file, 'a') { |f| f.write("\n" + css.join()) } - end + puts "cp -r " + source + "/. " + dest + cp_r(source + "/.", dest) # Build Epydoc for Python puts "Moving to python directory and building epydoc." @@ -70,11 +71,11 @@ puts "Moving back into docs dir." cd("../docs") - puts "echo making directory pyspark" - mkdir_p "pyspark" + puts "Making directory api/python" + mkdir_p "api/python" - puts "cp -r ../python/docs/. api/pyspark" - cp_r("../python/docs/.", "api/pyspark") + puts "cp -r ../python/docs/. api/python" + cp_r("../python/docs/.", "api/python") cd("..") end diff --git a/docs/api.md b/docs/api.md index 91c8e51d26c15..03460383335e8 100644 --- a/docs/api.md +++ b/docs/api.md @@ -1,13 +1,10 @@ --- layout: global -title: Spark API documentation (Scaladoc) +title: Spark API Documentation --- -Here you can find links to the Scaladoc generated for the Spark sbt subprojects. If the following links don't work, try running `sbt/sbt doc` from the Spark project home directory. +Here you can API docs for Spark and its submodules. -- [Spark](api/core/index.html) -- [Spark Examples](api/examples/index.html) -- [Spark Streaming](api/streaming/index.html) -- [Bagel](api/bagel/index.html) -- [GraphX](api/graphx/index.html) -- [PySpark](api/pyspark/index.html) +- [Spark Scala API (Scaladoc)](api/scala/index.html) +- [Spark Java API (Javadoc)](api/java/index.html) +- [Spark Python API (Epydoc)](api/python/index.html) diff --git a/docs/configuration.md b/docs/configuration.md index 5a4abca2646b6..e7e1dd56cf124 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -6,7 +6,7 @@ title: Spark Configuration Spark provides three locations to configure the system: * [Spark properties](#spark-properties) control most application parameters and can be set by passing - a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) object to SparkContext, or through Java + a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object to SparkContext, or through Java system properties. * [Environment variables](#environment-variables) can be used to set per-machine settings, such as the IP address, through the `conf/spark-env.sh` script on each node. @@ -16,7 +16,7 @@ Spark provides three locations to configure the system: # Spark Properties Spark properties control most application settings and are configured separately for each application. -The preferred way to set them is by passing a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) +The preferred way to set them is by passing a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) class to your SparkContext constructor. Alternatively, Spark will also load them from Java system properties, for compatibility with old versions of Spark. @@ -53,7 +53,7 @@ there are at least five properties that you will commonly want to control: in serialized form. The default of Java serialization works with any Serializable Java object but is quite slow, so we recommend using org.apache.spark.serializer.KryoSerializer and configuring Kryo serialization when speed is necessary. Can be any subclass of - org.apache.spark.Serializer. + org.apache.spark.Serializer.
      @@ -62,7 +62,7 @@ there are at least five properties that you will commonly want to control: diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 1238e3e0a4e7d..07be8ba58efa3 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -17,7 +17,7 @@ title: GraphX Programming Guide # Overview GraphX is the new (alpha) Spark API for graphs and graph-parallel computation. At a high-level, -GraphX extends the Spark [RDD](api/core/index.html#org.apache.spark.rdd.RDD) by introducing the +GraphX extends the Spark [RDD](api/scala/index.html#org.apache.spark.rdd.RDD) by introducing the [Resilient Distributed Property Graph](#property_graph): a directed multigraph with properties attached to each vertex and edge. To support graph computation, GraphX exposes a set of fundamental operators (e.g., [subgraph](#structural_operators), [joinVertices](#join_operators), and @@ -82,7 +82,7 @@ Prior to the release of GraphX, graph computation in Spark was expressed using B implementation of Pregel. GraphX improves upon Bagel by exposing a richer property graph API, a more streamlined version of the Pregel abstraction, and system optimizations to improve performance and reduce memory overhead. While we plan to eventually deprecate Bagel, we will continue to -support the [Bagel API](api/bagel/index.html#org.apache.spark.bagel.package) and +support the [Bagel API](api/scala/index.html#org.apache.spark.bagel.package) and [Bagel programming guide](bagel-programming-guide.html). However, we encourage Bagel users to explore the new GraphX API and comment on issues that may complicate the transition from Bagel. @@ -103,7 +103,7 @@ getting started with Spark refer to the [Spark Quick Start Guide](quick-start.ht # The Property Graph -The [property graph](api/graphx/index.html#org.apache.spark.graphx.Graph) is a directed multigraph +The [property graph](api/scala/index.html#org.apache.spark.graphx.Graph) is a directed multigraph with user defined objects attached to each vertex and edge. A directed multigraph is a directed graph with potentially multiple parallel edges sharing the same source and destination vertex. The ability to support parallel edges simplifies modeling scenarios where there can be multiple @@ -179,7 +179,7 @@ val userGraph: Graph[(String, String), String] There are numerous ways to construct a property graph from raw files, RDDs, and even synthetic generators and these are discussed in more detail in the section on [graph builders](#graph_builders). Probably the most general method is to use the -[Graph object](api/graphx/index.html#org.apache.spark.graphx.Graph$). For example the following +[Graph object](api/scala/index.html#org.apache.spark.graphx.Graph$). For example the following code constructs a graph from a collection of RDDs: {% highlight scala %} @@ -203,7 +203,7 @@ In the above example we make use of the [`Edge`][Edge] case class. Edges have a `dstId` corresponding to the source and destination vertex identifiers. In addition, the `Edge` class has an `attr` member which stores the edge property. -[Edge]: api/graphx/index.html#org.apache.spark.graphx.Edge +[Edge]: api/scala/index.html#org.apache.spark.graphx.Edge We can deconstruct a graph into the respective vertex and edge views by using the `graph.vertices` and `graph.edges` members respectively. @@ -229,7 +229,7 @@ The triplet view logically joins the vertex and edge properties yielding an `RDD[EdgeTriplet[VD, ED]]` containing instances of the [`EdgeTriplet`][EdgeTriplet] class. This *join* can be expressed in the following SQL expression: -[EdgeTriplet]: api/graphx/index.html#org.apache.spark.graphx.EdgeTriplet +[EdgeTriplet]: api/scala/index.html#org.apache.spark.graphx.EdgeTriplet {% highlight sql %} SELECT src.id, dst.id, src.attr, e.attr, dst.attr @@ -270,8 +270,8 @@ core operators are defined in [`GraphOps`][GraphOps]. However, thanks to Scala operators in `GraphOps` are automatically available as members of `Graph`. For example, we can compute the in-degree of each vertex (defined in `GraphOps`) by the following: -[Graph]: api/graphx/index.html#org.apache.spark.graphx.Graph -[GraphOps]: api/graphx/index.html#org.apache.spark.graphx.GraphOps +[Graph]: api/scala/index.html#org.apache.spark.graphx.Graph +[GraphOps]: api/scala/index.html#org.apache.spark.graphx.GraphOps {% highlight scala %} val graph: Graph[(String, String), String] @@ -382,7 +382,7 @@ val newGraph = Graph(newVertices, graph.edges) val newGraph = graph.mapVertices((id, attr) => mapUdf(id, attr)) {% endhighlight %} -[Graph.mapVertices]: api/graphx/index.html#org.apache.spark.graphx.Graph@mapVertices[VD2]((VertexId,VD)⇒VD2)(ClassTag[VD2]):Graph[VD2,ED] +[Graph.mapVertices]: api/scala/index.html#org.apache.spark.graphx.Graph@mapVertices[VD2]((VertexId,VD)⇒VD2)(ClassTag[VD2]):Graph[VD2,ED] These operators are often used to initialize the graph for a particular computation or project away unnecessary properties. For example, given a graph with the out-degrees as the vertex properties @@ -419,7 +419,7 @@ This can be useful when, for example, trying to compute the inverse PageRank. B operation does not modify vertex or edge properties or change the number of edges, it can be implemented efficiently without data-movement or duplication. -[Graph.reverse]: api/graphx/index.html#org.apache.spark.graphx.Graph@reverse:Graph[VD,ED] +[Graph.reverse]: api/scala/index.html#org.apache.spark.graphx.Graph@reverse:Graph[VD,ED] The [`subgraph`][Graph.subgraph] operator takes vertex and edge predicates and returns the graph containing only the vertices that satisfy the vertex predicate (evaluate to true) and edges that @@ -427,7 +427,7 @@ satisfy the edge predicate *and connect vertices that satisfy the vertex predica operator can be used in number of situations to restrict the graph to the vertices and edges of interest or eliminate broken links. For example in the following code we remove broken links: -[Graph.subgraph]: api/graphx/index.html#org.apache.spark.graphx.Graph@subgraph((EdgeTriplet[VD,ED])⇒Boolean,(VertexId,VD)⇒Boolean):Graph[VD,ED] +[Graph.subgraph]: api/scala/index.html#org.apache.spark.graphx.Graph@subgraph((EdgeTriplet[VD,ED])⇒Boolean,(VertexId,VD)⇒Boolean):Graph[VD,ED] {% highlight scala %} // Create an RDD for the vertices @@ -467,7 +467,7 @@ vertices and edges that are also found in the input graph. This can be used in example, we might run connected components using the graph with missing vertices and then restrict the answer to the valid subgraph. -[Graph.mask]: api/graphx/index.html#org.apache.spark.graphx.Graph@mask[VD2,ED2](Graph[VD2,ED2])(ClassTag[VD2],ClassTag[ED2]):Graph[VD,ED] +[Graph.mask]: api/scala/index.html#org.apache.spark.graphx.Graph@mask[VD2,ED2](Graph[VD2,ED2])(ClassTag[VD2],ClassTag[ED2]):Graph[VD,ED] {% highlight scala %} // Run Connected Components @@ -482,7 +482,7 @@ The [`groupEdges`][Graph.groupEdges] operator merges parallel edges (i.e., dupli pairs of vertices) in the multigraph. In many numerical applications, parallel edges can be *added* (their weights combined) into a single edge thereby reducing the size of the graph. -[Graph.groupEdges]: api/graphx/index.html#org.apache.spark.graphx.Graph@groupEdges((ED,ED)⇒ED):Graph[VD,ED] +[Graph.groupEdges]: api/scala/index.html#org.apache.spark.graphx.Graph@groupEdges((ED,ED)⇒ED):Graph[VD,ED] ## Join Operators @@ -506,7 +506,7 @@ returns a new graph with the vertex properties obtained by applying the user def to the result of the joined vertices. Vertices without a matching value in the RDD retain their original value. -[GraphOps.joinVertices]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@joinVertices[U](RDD[(VertexId,U)])((VertexId,VD,U)⇒VD)(ClassTag[U]):Graph[VD,ED] +[GraphOps.joinVertices]: api/scala/index.html#org.apache.spark.graphx.GraphOps@joinVertices[U](RDD[(VertexId,U)])((VertexId,VD,U)⇒VD)(ClassTag[U]):Graph[VD,ED] > Note that if the RDD contains more than one value for a given vertex only one will be used. It > is therefore recommended that the input RDD be first made unique using the following which will @@ -525,7 +525,7 @@ property type. Because not all vertices may have a matching value in the input function takes an `Option` type. For example, we can setup a graph for PageRank by initializing vertex properties with their `outDegree`. -[Graph.outerJoinVertices]: api/graphx/index.html#org.apache.spark.graphx.Graph@outerJoinVertices[U,VD2](RDD[(VertexId,U)])((VertexId,VD,Option[U])⇒VD2)(ClassTag[U],ClassTag[VD2]):Graph[VD2,ED] +[Graph.outerJoinVertices]: api/scala/index.html#org.apache.spark.graphx.Graph@outerJoinVertices[U,VD2](RDD[(VertexId,U)])((VertexId,VD,Option[U])⇒VD2)(ClassTag[U],ClassTag[VD2]):Graph[VD2,ED] {% highlight scala %} @@ -559,7 +559,7 @@ PageRank Value, shortest path to the source, and smallest reachable vertex id). ### Map Reduce Triplets (mapReduceTriplets) -[Graph.mapReduceTriplets]: api/graphx/index.html#org.apache.spark.graphx.Graph@mapReduceTriplets[A](mapFunc:org.apache.spark.graphx.EdgeTriplet[VD,ED]=>Iterator[(org.apache.spark.graphx.VertexId,A)],reduceFunc:(A,A)=>A,activeSetOpt:Option[(org.apache.spark.graphx.VertexRDD[_],org.apache.spark.graphx.EdgeDirection)])(implicitevidence$10:scala.reflect.ClassTag[A]):org.apache.spark.graphx.VertexRDD[A] +[Graph.mapReduceTriplets]: api/scala/index.html#org.apache.spark.graphx.Graph@mapReduceTriplets[A](mapFunc:org.apache.spark.graphx.EdgeTriplet[VD,ED]=>Iterator[(org.apache.spark.graphx.VertexId,A)],reduceFunc:(A,A)=>A,activeSetOpt:Option[(org.apache.spark.graphx.VertexRDD[_],org.apache.spark.graphx.EdgeDirection)])(implicitevidence$10:scala.reflect.ClassTag[A]):org.apache.spark.graphx.VertexRDD[A] The core (heavily optimized) aggregation primitive in GraphX is the [`mapReduceTriplets`][Graph.mapReduceTriplets] operator: @@ -665,8 +665,8 @@ attributes at each vertex. This can be easily accomplished using the [`collectNeighborIds`][GraphOps.collectNeighborIds] and the [`collectNeighbors`][GraphOps.collectNeighbors] operators. -[GraphOps.collectNeighborIds]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@collectNeighborIds(EdgeDirection):VertexRDD[Array[VertexId]] -[GraphOps.collectNeighbors]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@collectNeighbors(EdgeDirection):VertexRDD[Array[(VertexId,VD)]] +[GraphOps.collectNeighborIds]: api/scala/index.html#org.apache.spark.graphx.GraphOps@collectNeighborIds(EdgeDirection):VertexRDD[Array[VertexId]] +[GraphOps.collectNeighbors]: api/scala/index.html#org.apache.spark.graphx.GraphOps@collectNeighbors(EdgeDirection):VertexRDD[Array[(VertexId,VD)]] {% highlight scala %} @@ -685,7 +685,7 @@ class GraphOps[VD, ED] { In Spark, RDDs are not persisted in memory by default. To avoid recomputation, they must be explicitly cached when using them multiple times (see the [Spark Programming Guide][RDD Persistence]). Graphs in GraphX behave the same way. **When using a graph multiple times, make sure to call [`Graph.cache()`][Graph.cache] on it first.** [RDD Persistence]: scala-programming-guide.html#rdd-persistence -[Graph.cache]: api/graphx/index.html#org.apache.spark.graphx.Graph@cache():Graph[VD,ED] +[Graph.cache]: api/scala/index.html#org.apache.spark.graphx.Graph@cache():Graph[VD,ED] In iterative computations, *uncaching* may also be necessary for best performance. By default, cached RDDs and graphs will remain in memory until memory pressure forces them to be evicted in LRU order. For iterative computation, intermediate results from previous iterations will fill up the cache. Though they will eventually be evicted, the unnecessary data stored in memory will slow down garbage collection. It would be more efficient to uncache intermediate results as soon as they are no longer necessary. This involves materializing (caching and forcing) a graph or RDD every iteration, uncaching all other datasets, and only using the materialized dataset in future iterations. However, because graphs are composed of multiple RDDs, it can be difficult to unpersist them correctly. **For iterative computation we recommend using the Pregel API, which correctly unpersists intermediate results.** @@ -716,7 +716,7 @@ messages remaining. The following is the type signature of the [Pregel operator][GraphOps.pregel] as well as a *sketch* of its implementation (note calls to graph.cache have been removed): -[GraphOps.pregel]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@pregel[A](A,Int,EdgeDirection)((VertexId,VD,A)⇒VD,(EdgeTriplet[VD,ED])⇒Iterator[(VertexId,A)],(A,A)⇒A)(ClassTag[A]):Graph[VD,ED] +[GraphOps.pregel]: api/scala/index.html#org.apache.spark.graphx.GraphOps@pregel[A](A,Int,EdgeDirection)((VertexId,VD,A)⇒VD,(EdgeTriplet[VD,ED])⇒Iterator[(VertexId,A)],(A,A)⇒A)(ClassTag[A]):Graph[VD,ED] {% highlight scala %} class GraphOps[VD, ED] { @@ -840,12 +840,12 @@ object Graph { [`Graph.fromEdgeTuples`][Graph.fromEdgeTuples] allows creating a graph from only an RDD of edge tuples, assigning the edges the value 1, and automatically creating any vertices mentioned by edges and assigning them the default value. It also supports deduplicating the edges; to deduplicate, pass `Some` of a [`PartitionStrategy`][PartitionStrategy] as the `uniqueEdges` parameter (for example, `uniqueEdges = Some(PartitionStrategy.RandomVertexCut)`). A partition strategy is necessary to colocate identical edges on the same partition so they can be deduplicated. -[PartitionStrategy]: api/graphx/index.html#org.apache.spark.graphx.PartitionStrategy$ +[PartitionStrategy]: api/scala/index.html#org.apache.spark.graphx.PartitionStrategy$ -[GraphLoader.edgeListFile]: api/graphx/index.html#org.apache.spark.graphx.GraphLoader$@edgeListFile(SparkContext,String,Boolean,Int):Graph[Int,Int] -[Graph.apply]: api/graphx/index.html#org.apache.spark.graphx.Graph$@apply[VD,ED](RDD[(VertexId,VD)],RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED] -[Graph.fromEdgeTuples]: api/graphx/index.html#org.apache.spark.graphx.Graph$@fromEdgeTuples[VD](RDD[(VertexId,VertexId)],VD,Option[PartitionStrategy])(ClassTag[VD]):Graph[VD,Int] -[Graph.fromEdges]: api/graphx/index.html#org.apache.spark.graphx.Graph$@fromEdges[VD,ED](RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED] +[GraphLoader.edgeListFile]: api/scala/index.html#org.apache.spark.graphx.GraphLoader$@edgeListFile(SparkContext,String,Boolean,Int):Graph[Int,Int] +[Graph.apply]: api/scala/index.html#org.apache.spark.graphx.Graph$@apply[VD,ED](RDD[(VertexId,VD)],RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED] +[Graph.fromEdgeTuples]: api/scala/index.html#org.apache.spark.graphx.Graph$@fromEdgeTuples[VD](RDD[(VertexId,VertexId)],VD,Option[PartitionStrategy])(ClassTag[VD]):Graph[VD,Int] +[Graph.fromEdges]: api/scala/index.html#org.apache.spark.graphx.Graph$@fromEdges[VD,ED](RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED] # Vertex and Edge RDDs @@ -913,7 +913,7 @@ of the various partitioning strategies defined in [`PartitionStrategy`][Partitio each partition, edge attributes and adjacency structure, are stored separately enabling maximum reuse when changing attribute values. -[PartitionStrategy]: api/graphx/index.html#org.apache.spark.graphx.PartitionStrategy +[PartitionStrategy]: api/scala/index.html#org.apache.spark.graphx.PartitionStrategy The three additional functions exposed by the `EdgeRDD` are: {% highlight scala %} @@ -952,7 +952,7 @@ the [`Graph.partitionBy`][Graph.partitionBy] operator. The default partitioning the initial partitioning of the edges as provided on graph construction. However, users can easily switch to 2D-partitioning or other heuristics included in GraphX. -[Graph.partitionBy]: api/graphx/index.html#org.apache.spark.graphx.Graph$@partitionBy(partitionStrategy:org.apache.spark.graphx.PartitionStrategy):org.apache.spark.graphx.Graph[VD,ED] +[Graph.partitionBy]: api/scala/index.html#org.apache.spark.graphx.Graph$@partitionBy(partitionStrategy:org.apache.spark.graphx.PartitionStrategy):org.apache.spark.graphx.Graph[VD,ED]

      @@ -81,7 +81,7 @@ interface has a single abstract method, `call()`, that must be implemented. ## Storage Levels RDD [storage level](scala-programming-guide.html#rdd-persistence) constants, such as `MEMORY_AND_DISK`, are -declared in the [org.apache.spark.api.java.StorageLevels](api/core/index.html#org.apache.spark.api.java.StorageLevels) class. To +declared in the [org.apache.spark.api.java.StorageLevels](api/java/index.html?org/apache/spark/api/java/StorageLevels.html) class. To define your own storage level, you can use StorageLevels.create(...). # Other Features @@ -101,11 +101,11 @@ the following changes: classes to interfaces. This means that concrete implementations of these `Function` classes will need to use `implements` rather than `extends`. * Certain transformation functions now have multiple versions depending - on the return type. In Spark core, the map functions (map, flatMap, - mapPartitons) have type-specific versions, e.g. - [`mapToPair`](api/core/index.html#org.apache.spark.api.java.JavaRDD@mapToPair[K2,V2](f:org.apache.spark.api.java.function.PairFunction[T,K2,V2]):org.apache.spark.api.java.JavaPairRDD[K2,V2]) - and [`mapToDouble`](api/core/index.html#org.apache.spark.api.java.JavaRDD@mapToDouble[R](f:org.apache.spark.api.java.function.DoubleFunction[T]):org.apache.spark.api.java.JavaDoubleRDD). - Spark Streaming also uses the same approach, e.g. [`transformToPair`](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaDStream@transformToPair[K2,V2](transformFunc:org.apache.spark.api.java.function.Function[R,org.apache.spark.api.java.JavaPairRDD[K2,V2]]):org.apache.spark.streaming.api.java.JavaPairDStream[K2,V2]). + on the return type. In Spark core, the map functions (`map`, `flatMap`, and + `mapPartitons`) have type-specific versions, e.g. + [`mapToPair`](api/java/org/apache/spark/api/java/JavaRDDLike.html#mapToPair(org.apache.spark.api.java.function.PairFunction)) + and [`mapToDouble`](api/java/org/apache/spark/api/java/JavaRDDLike.html#mapToDouble(org.apache.spark.api.java.function.DoubleFunction)). + Spark Streaming also uses the same approach, e.g. [`transformToPair`](api/java/org/apache/spark/streaming/api/java/JavaDStreamLike.html#transformToPair(org.apache.spark.api.java.function.Function)). # Example @@ -205,16 +205,9 @@ JavaPairRDD counts = lines.flatMapToPair( There is no performance difference between these approaches; the choice is just a matter of style. -# Javadoc - -We currently provide documentation for the Java API as Scaladoc, in the -[`org.apache.spark.api.java` package](api/core/index.html#org.apache.spark.api.java.package), because -some of the classes are implemented in Scala. It is important to note that the types and function -definitions show Scala syntax (for example, `def reduce(func: Function2[T, T]): T` instead of -`T reduce(Function2 func)`). In addition, the Scala `trait` modifier is used for Java -interface classes. We hope to generate documentation with Java-style syntax in the future to -avoid these quirks. +# API Docs +[API documentation](api/java/index.html) for Spark in Java is available in Javadoc format. # Where to Go from Here diff --git a/docs/js/main.js b/docs/js/main.js index 0bd2286cced19..59055467110bf 100755 --- a/docs/js/main.js +++ b/docs/js/main.js @@ -73,8 +73,26 @@ function viewSolution() { }); } +// A script to fix internal hash links because we have an overlapping top bar. +// Based on https://github.com/twitter/bootstrap/issues/193#issuecomment-2281510 +function maybeScrollToHash() { + console.log("HERE"); + if (window.location.hash && $(window.location.hash).length) { + console.log("HERE2", $(window.location.hash), $(window.location.hash).offset().top); + var newTop = $(window.location.hash).offset().top - 57; + $(window).scrollTop(newTop); + } +} $(function() { codeTabs(); viewSolution(); + + $(window).bind('hashchange', function() { + maybeScrollToHash(); + }); + + // Scroll now too in case we had opened the page on a hash, but wait a bit because some browsers + // will try to do *their* initial scroll after running the onReady handler. + $(window).load(function() { setTimeout(function() { maybeScrollToHash(); }, 25); }); }); diff --git a/docs/mllib-classification-regression.md b/docs/mllib-classification-regression.md index 2c42f60c2ecce..2e0fa093dccaa 100644 --- a/docs/mllib-classification-regression.md +++ b/docs/mllib-classification-regression.md @@ -316,26 +316,26 @@ For each of them, we support all 3 possible regularizations (none, L1 or L2). Available algorithms for binary classification: -* [SVMWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.SVMWithSGD) -* [LogisticRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) +* [SVMWithSGD](api/scala/index.html#org.apache.spark.mllib.classification.SVMWithSGD) +* [LogisticRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) Available algorithms for linear regression: -* [LinearRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD) -* [RidgeRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) -* [LassoWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.LassoWithSGD) +* [LinearRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD) +* [RidgeRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) +* [LassoWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.LassoWithSGD) Behind the scenes, all above methods use the SGD implementation from the gradient descent primitive in MLlib, see the optimization part: -* [GradientDescent](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) +* [GradientDescent](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) #### Tree-based Methods The decision tree algorithm supports binary classification and regression: -* [DecisionTee](api/mllib/index.html#org.apache.spark.mllib.tree.DecisionTree) +* [DecisionTee](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) # Usage in Scala diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 50a8671560737..0359c67157168 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -33,7 +33,7 @@ a given dataset, the algorithm returns the best clustering result). Available algorithms for clustering: -* [KMeans](api/mllib/index.html#org.apache.spark.mllib.clustering.KMeans) +* [KMeans](api/scala/index.html#org.apache.spark.mllib.clustering.KMeans) diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index aa22f67b303ce..2f1f5f3856efe 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -42,7 +42,7 @@ for an item. Available algorithms for collaborative filtering: -* [ALS](api/mllib/index.html#org.apache.spark.mllib.recommendation.ALS) +* [ALS](api/scala/index.html#org.apache.spark.mllib.recommendation.ALS) # Usage in Scala diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 4236b0c8b6c99..0963a99881c9d 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -36,15 +36,15 @@ The following links provide a detailed explanation of the methods and usage exam # Data Types Most MLlib algorithms operate on RDDs containing vectors. In Java and Scala, the -[Vector](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) class is used to +[Vector](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) class is used to represent vectors. You can create either dense or sparse vectors using the -[Vectors](api/mllib/index.html#org.apache.spark.mllib.linalg.Vectors$) factory. +[Vectors](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) factory. In Python, MLlib can take the following vector types: * [NumPy](http://www.numpy.org) arrays * Standard Python lists (e.g. `[1, 2, 3]`) -* The MLlib [SparseVector](api/pyspark/pyspark.mllib.linalg.SparseVector-class.html) class +* The MLlib [SparseVector](api/python/pyspark.mllib.linalg.SparseVector-class.html) class * [SciPy sparse matrices](http://docs.scipy.org/doc/scipy/reference/sparse.html) For efficiency, we recommend using NumPy arrays over lists, and using the @@ -52,8 +52,8 @@ For efficiency, we recommend using NumPy arrays over lists, and using the for SciPy matrices, or MLlib's own SparseVector class. Several other simple data types are used throughout the library, e.g. the LabeledPoint -class ([Java/Scala](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint), -[Python](api/pyspark/pyspark.mllib.regression.LabeledPoint-class.html)) for labeled data. +class ([Java/Scala](api/scala/index.html#org.apache.spark.mllib.regression.LabeledPoint), +[Python](api/python/pyspark.mllib.regression.LabeledPoint-class.html)) for labeled data. # Dependencies MLlib uses the [jblas](https://github.com/mikiobraun/jblas) linear algebra library, which itself diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index 396b98d52a994..c79cc3d944f32 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -95,12 +95,12 @@ As an alternative to just use the subgradient `$R'(\wv)$` of the regularizer in direction, an improved update for some cases can be obtained by using the proximal operator instead. For the L1-regularizer, the proximal operator is given by soft thresholding, as implemented in -[L1Updater](api/mllib/index.html#org.apache.spark.mllib.optimization.L1Updater). +[L1Updater](api/scala/index.html#org.apache.spark.mllib.optimization.L1Updater). ## Update Schemes for Distributed SGD The SGD implementation in -[GradientDescent](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) uses +[GradientDescent](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) uses a simple (distributed) sampling of the data examples. We recall that the loss part of the optimization problem `$\eqref{eq:regPrimal}$` is `$\frac1n \sum_{i=1}^n L(\wv;\x_i,y_i)$`, and therefore `$\frac1n \sum_{i=1}^n L'_{\wv,i}$` would @@ -138,7 +138,7 @@ are developed, see the section for example. The SGD method -[GradientDescent.runMiniBatchSGD](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) +[GradientDescent.runMiniBatchSGD](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) has the following parameters: * `gradient` is a class that computes the stochastic gradient of the function @@ -161,6 +161,6 @@ each iteration, to compute the gradient direction. Available algorithms for gradient descent: -* [GradientDescent.runMiniBatchSGD](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) +* [GradientDescent.runMiniBatchSGD](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 39de603b29f87..98233bf556b79 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -134,7 +134,7 @@ Files listed here will be added to the `PYTHONPATH` and shipped to remote worker Code dependencies can be added to an existing SparkContext using its `addPyFile()` method. You can set [configuration properties](configuration.html#spark-properties) by passing a -[SparkConf](api/pyspark/pyspark.conf.SparkConf-class.html) object to SparkContext: +[SparkConf](api/python/pyspark.conf.SparkConf-class.html) object to SparkContext: {% highlight python %} from pyspark import SparkConf, SparkContext @@ -147,7 +147,7 @@ sc = SparkContext(conf = conf) # API Docs -[API documentation](api/pyspark/index.html) for PySpark is available as Epydoc. +[API documentation](api/python/index.html) for PySpark is available as Epydoc. Many of the methods also contain [doctests](http://docs.python.org/2/library/doctest.html) that provide additional usage examples. # Libraries diff --git a/docs/quick-start.md b/docs/quick-start.md index 6b4f4ba4254a2..68afa6e1bff95 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -138,7 +138,9 @@ Spark README. Note that you'll need to replace YOUR_SPARK_HOME with the location installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the program. -We pass the SparkContext constructor a SparkConf object which contains information about our +We pass the SparkContext constructor a +[SparkConf](api/scala/index.html#org.apache.spark.SparkConf) +object which contains information about our application. We also call sc.addJar to make sure that when our application is launched in cluster mode, the jar file containing it will be shipped automatically to worker nodes. @@ -327,4 +329,4 @@ Congratulations on running your first Spark application! * For an in-depth overview of the API see "Programming Guides" menu section. * For running applications on a cluster head to the [deployment overview](cluster-overview.html). -* For configuration options available to Spark applications see the [configuration page](configuration.html). \ No newline at end of file +* For configuration options available to Spark applications see the [configuration page](configuration.html). diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 4431da0721ac7..a3171709ff05b 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -147,7 +147,7 @@ All transformations in Spark are lazy, in that they do not compute their By default, each transformed RDD is recomputed each time you run an action on it. However, you may also *persist* an RDD in memory using the `persist` (or `cache`) method, in which case Spark will keep the elements around on the cluster for much faster access the next time you query it. There is also support for persisting datasets on disk, or replicated across the cluster. The next section in this document describes these options. -The following tables list the transformations and actions currently supported (see also the [RDD API doc](api/core/index.html#org.apache.spark.rdd.RDD) for details): +The following tables list the transformations and actions currently supported (see also the [RDD API doc](api/scala/index.html#org.apache.spark.rdd.RDD) for details): ### Transformations @@ -216,7 +216,7 @@ The following tables list the transformations and actions currently supported (s
      Stage IdPool NameDescriptionSubmittedDurationTasks: Succeeded/TotalShuffle ReadShuffle Write
      {s.stageId} - - {poolName.get} - - {makeDescription(s)}{submissionTime}{formattedDuration} - {makeProgressBar(startedTasks, completedTasks, failedTasks, totalTasks)} + {s.stageId} + + {poolName.get} + {shuffleRead}{shuffleWrite}
      {makeDescription(s)}{submissionTime}{formattedDuration} + {makeProgressBar(startedTasks, completedTasks, failedTasks, totalTasks)} + {shuffleRead}{shuffleWrite}
      Failure Reason{s.failureReason.getOrElse("")}
      If you use Kryo serialization, set this class to register your custom classes with Kryo. It should be set to a class that extends - KryoRegistrator. + KryoRegistrator. See the tuning guide for more details.
      ClassFunction Type
      -A complete list of transformations is available in the [RDD API doc](api/core/index.html#org.apache.spark.rdd.RDD). +A complete list of transformations is available in the [RDD API doc](api/scala/index.html#org.apache.spark.rdd.RDD). ### Actions @@ -264,7 +264,7 @@ A complete list of transformations is available in the [RDD API doc](api/core/in -A complete list of actions is available in the [RDD API doc](api/core/index.html#org.apache.spark.rdd.RDD). +A complete list of actions is available in the [RDD API doc](api/scala/index.html#org.apache.spark.rdd.RDD). ## RDD Persistence @@ -283,7 +283,7 @@ In addition, each RDD can be stored using a different *storage level*, allowing persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), or replicate it across nodes, or store the data in off-heap memory in [Tachyon](http://tachyon-project.org/). These levels are chosen by passing a -[`org.apache.spark.storage.StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel) +[`org.apache.spark.storage.StorageLevel`](api/scala/index.html#org.apache.spark.storage.StorageLevel) object to `persist()`. The `cache()` method is a shorthand for using the default storage level, which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The complete set of available storage levels is: @@ -355,7 +355,7 @@ waiting to recompute a lost partition. If you want to define your own storage level (say, with replication factor of 3 instead of 2), then use the function factor method `apply()` of the -[`StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel$) singleton object. +[`StorageLevel`](api/scala/index.html#org.apache.spark.storage.StorageLevel$) singleton object. Spark has a block manager inside the Executors that let you chose memory, disk, or off-heap. The latter is for storing RDDs off-heap outside the Executor JVM on top of the memory management system diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 8e98cc0c80a34..e25379bd7614d 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -14,8 +14,8 @@ title: Spark SQL Programming Guide Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using Spark. At the core of this component is a new type of RDD, -[SchemaRDD](api/sql/core/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed -[Row](api/sql/catalyst/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects along with +[SchemaRDD](api/scala/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed +[Row](api/scala/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects along with a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table in a traditional relational database. A SchemaRDD can be created from an existing RDD, parquet file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). @@ -27,8 +27,8 @@ file, or by running HiveQL against data stored in [Apache Hive](http://hive.apac
      Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using Spark. At the core of this component is a new type of RDD, -[JavaSchemaRDD](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed -[Row](api/sql/catalyst/index.html#org.apache.spark.sql.api.java.Row) objects along with +[JavaSchemaRDD](api/scala/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed +[Row](api/scala/index.html#org.apache.spark.sql.api.java.Row) objects along with a schema that describes the data types of each column in the row. A JavaSchemaRDD is similar to a table in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, parquet file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). @@ -38,8 +38,8 @@ file, or by running HiveQL against data stored in [Apache Hive](http://hive.apac Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using Spark. At the core of this component is a new type of RDD, -[SchemaRDD](api/pyspark/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed -[Row](api/pyspark/pyspark.sql.Row-class.html) objects along with +[SchemaRDD](api/python/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed +[Row](api/python/pyspark.sql.Row-class.html) objects along with a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table in a traditional relational database. A SchemaRDD can be created from an existing RDD, parquet file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). @@ -56,7 +56,7 @@ file, or by running HiveQL against data stored in [Apache Hive](http://hive.apac
      The entry point into all relational functionality in Spark is the -[SQLContext](api/sql/core/index.html#org.apache.spark.sql.SQLContext) class, or one of its +[SQLContext](api/scala/index.html#org.apache.spark.sql.SQLContext) class, or one of its descendants. To create a basic SQLContext, all you need is a SparkContext. {% highlight scala %} @@ -72,7 +72,7 @@ import sqlContext._
      The entry point into all relational functionality in Spark is the -[JavaSQLContext](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one +[JavaSQLContext](api/scala/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one of its descendants. To create a basic JavaSQLContext, all you need is a JavaSparkContext. {% highlight java %} @@ -85,7 +85,7 @@ JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(ctx);
      The entry point into all relational functionality in Spark is the -[SQLContext](api/pyspark/pyspark.sql.SQLContext-class.html) class, or one +[SQLContext](api/python/pyspark.sql.SQLContext-class.html) class, or one of its decedents. To create a basic SQLContext, all you need is a SparkContext. {% highlight python %} @@ -331,7 +331,7 @@ val teenagers = people.where('age >= 10).where('age <= 19).select('name) The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers prefixed with a tick (`'`). Implicit conversions turn these symbols into expressions that are evaluated by the SQL execution engine. A full list of the functions supported can be found in the -[ScalaDoc](api/sql/core/index.html#org.apache.spark.sql.SchemaRDD). +[ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index 3fb540c9fb560..3cfa4516cc12d 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -9,7 +9,7 @@ This guide shows the programming model and features by walking through a simple ### Writing a Simple Receiver -This starts with implementing [NetworkReceiver](api/streaming/index.html#org.apache.spark.streaming.dstream.NetworkReceiver). +This starts with implementing [NetworkReceiver](api/scala/index.html#org.apache.spark.streaming.dstream.NetworkReceiver). The following is a simple socket text-stream receiver. @@ -125,4 +125,4 @@ _A more comprehensive example is provided in the spark streaming examples_ ## References 1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html) -2.[NetworkReceiver](api/streaming/index.html#org.apache.spark.streaming.dstream.NetworkReceiver) +2.[NetworkReceiver](api/scala/index.html#org.apache.spark.streaming.dstream.NetworkReceiver) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index f9904d45013f6..946d6c4879cf3 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -40,7 +40,7 @@ Spark Streaming provides a high-level abstraction called *discretized stream* or which represents a continuous stream of data. DStreams can be created either from input data stream from sources such as Kafka and Flume, or by applying high-level operations on other DStreams. Internally, a DStream is represented as a sequence of -[RDDs](api/core/index.html#org.apache.spark.rdd.RDD). +[RDDs](api/scala/index.html#org.apache.spark.rdd.RDD). This guide shows you how to start writing Spark Streaming programs with DStreams. You can write Spark Streaming programs in Scala or Java, both of which are presented in this guide. You @@ -62,7 +62,7 @@ First, we import the names of the Spark Streaming classes, and some implicit conversions from StreamingContext into our environment, to add useful methods to other classes we need (like DStream). -[StreamingContext](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) is the +[StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) is the main entry point for all streaming functionality. {% highlight scala %} @@ -71,7 +71,7 @@ import org.apache.spark.streaming.StreamingContext._ {% endhighlight %} Then we create a -[StreamingContext](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) object. +[StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) object. Besides Spark's configuration, we specify that any DStream will be processed in 1 second batches. @@ -132,7 +132,7 @@ The complete code can be found in the Spark Streaming example
      First, we create a -[JavaStreamingContext](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) object, +[JavaStreamingContext](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) object, which is the main entry point for all streaming functionality. Besides Spark's configuration, we specify that any DStream would be processed in 1 second batches. @@ -168,7 +168,7 @@ JavaDStream words = lines.flatMap( generating multiple new records from each record in the source DStream. In this case, each line will be split into multiple words and the stream of words is represented as the `words` DStream. Note that we defined the transformation using a -[FlatMapFunction](api/core/index.html#org.apache.spark.api.java.function.FlatMapFunction) object. +[FlatMapFunction](api/scala/index.html#org.apache.spark.api.java.function.FlatMapFunction) object. As we will discover along the way, there are a number of such convenience classes in the Java API that help define DStream transformations. @@ -192,9 +192,9 @@ wordCounts.print(); // Print a few of the counts to the console {% endhighlight %} The `words` DStream is further mapped (one-to-one transformation) to a DStream of `(word, -1)` pairs, using a [PairFunction](api/core/index.html#org.apache.spark.api.java.function.PairFunction) +1)` pairs, using a [PairFunction](api/scala/index.html#org.apache.spark.api.java.function.PairFunction) object. Then, it is reduced to get the frequency of words in each batch of data, -using a [Function2](api/core/index.html#org.apache.spark.api.java.function.Function2) object. +using a [Function2](api/scala/index.html#org.apache.spark.api.java.function.Function2) object. Finally, `wordCounts.print()` will print a few of the counts generated every second. Note that when these lines are executed, Spark Streaming only sets up the computation it @@ -333,7 +333,7 @@ for the full list of supported sources and artifacts.
      To initialize a Spark Streaming program in Scala, a -[`StreamingContext`](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) +[`StreamingContext`](api/scala/index.html#org.apache.spark.streaming.StreamingContext) object has to be created, which is the main entry point of all Spark Streaming functionality. A `StreamingContext` object can be created by using @@ -344,7 +344,7 @@ new StreamingContext(master, appName, batchDuration, [sparkHome], [jars])
      To initialize a Spark Streaming program in Java, a -[`JavaStreamingContext`](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) +[`JavaStreamingContext`](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) object has to be created, which is the main entry point of all Spark Streaming functionality. A `JavaStreamingContext` object can be created by using @@ -431,8 +431,8 @@ and process any files created in that directory. Note that For more details on streams from files, Akka actors and sockets, see the API documentations of the relevant functions in -[StreamingContext](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) for -Scala and [JavaStreamingContext](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) +[StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) for +Scala and [JavaStreamingContext](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) for Java. Additional functionality for creating DStreams from sources such as Kafka, Flume, and Twitter @@ -802,10 +802,10 @@ output operators are defined: The complete list of DStream operations is available in the API documentation. For the Scala API, -see [DStream](api/streaming/index.html#org.apache.spark.streaming.dstream.DStream) -and [PairDStreamFunctions](api/streaming/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions). -For the Java API, see [JavaDStream](api/streaming/index.html#org.apache.spark.streaming.api.java.dstream.DStream) -and [JavaPairDStream](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaPairDStream). +see [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) +and [PairDStreamFunctions](api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions). +For the Java API, see [JavaDStream](api/scala/index.html#org.apache.spark.streaming.api.java.dstream.DStream) +and [JavaPairDStream](api/scala/index.html#org.apache.spark.streaming.api.java.JavaPairDStream). Specifically for the Java API, see [Spark's Java programming guide](java-programming-guide.html) for more information. @@ -881,7 +881,7 @@ Cluster resources maybe under-utilized if the number of parallel tasks used in a computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the -[`PairDStreamFunctions`](api/streaming/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions) +[`PairDStreamFunctions`](api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions) documentation), or set the [config property](configuration.html#spark-properties) `spark.default.parallelism` to change the default. @@ -925,7 +925,7 @@ A good approach to figure out the right batch size for your application is to te conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system is able to keep up with data rate, you can check the value of the end-to-end delay experienced by each processed batch (either look for "Total delay" in Spark driver log4j logs, or use the -[StreamingListener](api/streaming/index.html#org.apache.spark.streaming.scheduler.StreamingListener) +[StreamingListener](api/scala/index.html#org.apache.spark.streaming.scheduler.StreamingListener) interface). If the delay is maintained to be comparable to the batch size, then system is stable. Otherwise, if the delay is continuously increasing, it means that the system is unable to keep up and it @@ -952,7 +952,7 @@ exception saying so. ## Monitoring Besides Spark's in-built [monitoring capabilities](monitoring.html), the progress of a Spark Streaming program can also be monitored using the [StreamingListener] -(api/streaming/index.html#org.apache.spark.scheduler.StreamingListener) interface, +(api/scala/index.html#org.apache.spark.scheduler.StreamingListener) interface, which allows you to get statistics of batch processing times, queueing delays, and total end-to-end delays. Note that this is still an experimental API and it is likely to be improved upon (i.e., more information reported) in the future. @@ -965,9 +965,9 @@ in Spark Streaming applications and achieving more consistent batch processing t * **Default persistence level of DStreams**: Unlike RDDs, the default persistence level of DStreams serializes the data in memory (that is, -[StorageLevel.MEMORY_ONLY_SER](api/core/index.html#org.apache.spark.storage.StorageLevel$) for +[StorageLevel.MEMORY_ONLY_SER](api/scala/index.html#org.apache.spark.storage.StorageLevel$) for DStream compared to -[StorageLevel.MEMORY_ONLY](api/core/index.html#org.apache.spark.storage.StorageLevel$) for RDDs). +[StorageLevel.MEMORY_ONLY](api/scala/index.html#org.apache.spark.storage.StorageLevel$) for RDDs). Even though keeping the data serialized incurs higher serialization/deserialization overheads, it significantly reduces GC pauses. @@ -1244,15 +1244,15 @@ and output 30 after recovery. # Where to Go from Here * API documentation - - Main docs of StreamingContext and DStreams in [Scala](api/streaming/index.html#org.apache.spark.streaming.package) - and [Java](api/streaming/index.html#org.apache.spark.streaming.api.java.package) + - Main docs of StreamingContext and DStreams in [Scala](api/scala/index.html#org.apache.spark.streaming.package) + and [Java](api/scala/index.html#org.apache.spark.streaming.api.java.package) - Additional docs for - [Kafka](api/external/kafka/index.html#org.apache.spark.streaming.kafka.KafkaUtils$), - [Flume](api/external/flume/index.html#org.apache.spark.streaming.flume.FlumeUtils$), - [Twitter](api/external/twitter/index.html#org.apache.spark.streaming.twitter.TwitterUtils$), - [ZeroMQ](api/external/zeromq/index.html#org.apache.spark.streaming.zeromq.ZeroMQUtils$), and - [MQTT](api/external/mqtt/index.html#org.apache.spark.streaming.mqtt.MQTTUtils$) + [Kafka](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$), + [Flume](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$), + [Twitter](api/scala/index.html#org.apache.spark.streaming.twitter.TwitterUtils$), + [ZeroMQ](api/scala/index.html#org.apache.spark.streaming.zeromq.ZeroMQUtils$), and + [MQTT](api/scala/index.html#org.apache.spark.streaming.mqtt.MQTTUtils$) * More examples in [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples) and [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/streaming/examples) -* [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) describing Spark Streaming +* [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) describing Spark Streaming. diff --git a/docs/tuning.md b/docs/tuning.md index cc069f0e84b9c..78e10770a80b6 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -48,7 +48,7 @@ Spark automatically includes Kryo serializers for the many commonly-used core Sc in the AllScalaRegistrar from the [Twitter chill](https://github.com/twitter/chill) library. To register your own custom classes with Kryo, create a public class that extends -[`org.apache.spark.serializer.KryoRegistrator`](api/core/index.html#org.apache.spark.serializer.KryoRegistrator) and set the +[`org.apache.spark.serializer.KryoRegistrator`](api/scala/index.html#org.apache.spark.serializer.KryoRegistrator) and set the `spark.kryo.registrator` config property to point to it, as follows: {% highlight scala %} @@ -222,7 +222,7 @@ enough. Spark automatically sets the number of "map" tasks to run on each file a (though you can control it through optional parameters to `SparkContext.textFile`, etc), and for distributed "reduce" operations, such as `groupByKey` and `reduceByKey`, it uses the largest parent RDD's number of partitions. You can pass the level of parallelism as a second argument -(see the [`spark.PairRDDFunctions`](api/core/index.html#org.apache.spark.rdd.PairRDDFunctions) documentation), +(see the [`spark.PairRDDFunctions`](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions) documentation), or set the config property `spark.default.parallelism` to change the default. In general, we recommend 2-3 tasks per CPU core in your cluster. diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 33f9d644ca66d..f115f0d3a041c 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -23,6 +23,8 @@ import AssemblyKeys._ import scala.util.Properties import org.scalastyle.sbt.ScalastylePlugin.{Settings => ScalaStyleSettings} import com.typesafe.tools.mima.plugin.MimaKeys.previousArtifact +import sbtunidoc.Plugin._ +import UnidocKeys._ import scala.collection.JavaConversions._ @@ -31,6 +33,7 @@ import scala.collection.JavaConversions._ object SparkBuild extends Build { val SPARK_VERSION = "1.0.0-SNAPSHOT" + val SPARK_VERSION_SHORT = SPARK_VERSION.replaceAll("-SNAPSHOT", "") // Hadoop version to build against. For example, "1.0.4" for Apache releases, or // "2.0.0-mr1-cdh4.2.0" for Cloudera Hadoop. Note that these variables can be set @@ -184,12 +187,17 @@ object SparkBuild extends Build { // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), // Remove certain packages from Scaladoc - scalacOptions in (Compile,doc) := Seq("-groups", "-skip-packages", Seq( - "akka", - "org.apache.spark.network", - "org.apache.spark.deploy", - "org.apache.spark.util.collection" - ).mkString(":")), + scalacOptions in (Compile, doc) := Seq( + "-groups", + "-skip-packages", Seq( + "akka", + "org.apache.spark.api.python", + "org.apache.spark.network", + "org.apache.spark.deploy", + "org.apache.spark.util.collection" + ).mkString(":"), + "-doc-title", "Spark " + SPARK_VERSION_SHORT + " ScalaDoc" + ), // 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), @@ -283,7 +291,7 @@ object SparkBuild extends Build { publishMavenStyle in MavenCompile := true, publishLocal in MavenCompile <<= publishTask(publishLocalConfiguration in MavenCompile, deliverLocal), publishLocalBoth <<= Seq(publishLocal in MavenCompile, publishLocal).dependOn - ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings ++ ScalaStyleSettings + ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings ++ ScalaStyleSettings ++ genjavadocSettings val akkaVersion = "2.2.3-shaded-protobuf" val chillVersion = "0.3.1" @@ -349,15 +357,57 @@ object SparkBuild extends Build { libraryDependencies ++= maybeAvro ) - def rootSettings = sharedSettings ++ Seq( - publish := {} + // Create a colon-separate package list adding "org.apache.spark" in front of all of them, + // for easier specification of JavaDoc package groups + def packageList(names: String*): String = { + names.map(s => "org.apache.spark." + s).mkString(":") + } + + def rootSettings = sharedSettings ++ scalaJavaUnidocSettings ++ Seq( + publish := {}, + + unidocProjectFilter in (ScalaUnidoc, unidoc) := + inAnyProject -- inProjects(repl, examples, tools, yarn, yarnAlpha), + unidocProjectFilter in (JavaUnidoc, unidoc) := + inAnyProject -- inProjects(repl, examples, bagel, graphx, catalyst, tools, yarn, yarnAlpha), + + // Skip class names containing $ and some internal packages in Javadocs + unidocAllSources in (JavaUnidoc, unidoc) := { + (unidocAllSources in (JavaUnidoc, unidoc)).value + .map(_.filterNot(_.getName.contains("$"))) + .map(_.filterNot(_.getCanonicalPath.contains("akka"))) + .map(_.filterNot(_.getCanonicalPath.contains("deploy"))) + .map(_.filterNot(_.getCanonicalPath.contains("network"))) + .map(_.filterNot(_.getCanonicalPath.contains("executor"))) + .map(_.filterNot(_.getCanonicalPath.contains("python"))) + .map(_.filterNot(_.getCanonicalPath.contains("collection"))) + }, + + // Javadoc options: create a window title, and group key packages on index page + javacOptions in doc := Seq( + "-windowtitle", "Spark " + SPARK_VERSION_SHORT + " JavaDoc", + "-public", + "-group", "Core Java API", packageList("api.java", "api.java.function"), + "-group", "Spark Streaming", packageList( + "streaming.api.java", "streaming.flume", "streaming.kafka", + "streaming.mqtt", "streaming.twitter", "streaming.zeromq" + ), + "-group", "MLlib", packageList( + "mllib.classification", "mllib.clustering", "mllib.evaluation.binary", "mllib.linalg", + "mllib.linalg.distributed", "mllib.optimization", "mllib.rdd", "mllib.recommendation", + "mllib.regression", "mllib.stat", "mllib.tree", "mllib.tree.configuration", + "mllib.tree.impurity", "mllib.tree.model", "mllib.util" + ), + "-group", "Spark SQL", packageList("sql.api.java", "sql.hive.api.java"), + "-noqualifier", "java.lang" + ) ) def replSettings = sharedSettings ++ Seq( name := "spark-repl", - libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v ), - libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "jline" % v ), - libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-reflect" % v ) + libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v), + libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "jline" % v), + libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-reflect" % v) ) def examplesSettings = sharedSettings ++ Seq( diff --git a/project/plugins.sbt b/project/plugins.sbt index c25a25863d6ed..0cd16fd5bedd4 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -23,3 +23,4 @@ addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.0") +addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.0") diff --git a/python/epydoc.conf b/python/epydoc.conf index 081ed215ae60c..b73860bad8263 100644 --- a/python/epydoc.conf +++ b/python/epydoc.conf @@ -18,8 +18,8 @@ # # Information about the project. -name: PySpark -url: http://spark-project.org +name: Spark 1.0.0 Python API Docs +url: http://spark.apache.org # The list of modules to document. Modules can be named using # dotted names, module filenames, or package directory names. From b3e5366f696c463f1c2f033b0d5c7365e5d6b0f8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 21 Apr 2014 22:24:44 -0700 Subject: [PATCH 123/641] [Fix #274] Document + fix annotation usages ... so that we don't follow an unspoken set of forbidden rules for adding **@AlphaComponent**, **@DeveloperApi**, and **@Experimental** annotations in the code. In addition, this PR (1) removes unnecessary `:: * ::` tags, (2) adds missing `:: * ::` tags, and (3) removes annotations for internal APIs. Author: Andrew Or Closes #470 from andrewor14/annotations-fix and squashes the following commits: 92a7f42 [Andrew Or] Document + fix annotation usages --- core/src/main/scala/org/apache/spark/SparkContext.scala | 1 + .../org/apache/spark/annotation/AlphaComponent.java | 9 ++++++++- .../scala/org/apache/spark/annotation/DeveloperApi.java | 5 +++++ .../scala/org/apache/spark/annotation/Experimental.java | 5 +++++ core/src/main/scala/org/apache/spark/rdd/RDD.scala | 6 ------ .../scala/org/apache/spark/scheduler/SplitInfo.scala | 8 ++++++-- .../scala/org/apache/spark/mllib/clustering/KMeans.scala | 6 +++--- .../apache/spark/mllib/regression/RegressionModel.scala | 3 --- .../src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 -- .../src/main/scala/org/apache/spark/sql/SchemaRDD.scala | 2 +- .../scala/org/apache/spark/sql/hive/HiveContext.scala | 1 - 11 files changed, 29 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 7933d68d67d96..aa49f32ecdf62 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1110,6 +1110,7 @@ class SparkContext(config: SparkConf) extends Logging { } /** + * :: Experimental :: * Submit a job for execution and return a FutureJob holding the result. */ @Experimental diff --git a/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java b/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java index af01fb7cfbd04..db7b25c727d34 100644 --- a/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java +++ b/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java @@ -19,7 +19,14 @@ import java.lang.annotation.*; -/** A new component of Spark which may have unstable API's. */ +/** + * A new component of Spark which may have unstable API's. + * + * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first + * line of the comment must be ":: AlphaComponent ::" with no trailing blank line. This is because + * of the known issue that Scaladoc displays only either the annotation or the comment, whichever + * comes first. + */ @Retention(RetentionPolicy.RUNTIME) @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) diff --git a/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java b/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java index 5d546e7a63985..0ecef6db0e039 100644 --- a/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java +++ b/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java @@ -23,6 +23,11 @@ * A lower-level, unstable API intended for developers. * * Developer API's might change or be removed in minor versions of Spark. + * + * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first + * line of the comment must be ":: DeveloperApi ::" with no trailing blank line. This is because + * of the known issue that Scaladoc displays only either the annotation or the comment, whichever + * comes first. */ @Retention(RetentionPolicy.RUNTIME) @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, diff --git a/core/src/main/scala/org/apache/spark/annotation/Experimental.java b/core/src/main/scala/org/apache/spark/annotation/Experimental.java index 306b1418d8d0a..ff8120291455f 100644 --- a/core/src/main/scala/org/apache/spark/annotation/Experimental.java +++ b/core/src/main/scala/org/apache/spark/annotation/Experimental.java @@ -24,6 +24,11 @@ * * Experimental API's might change or be removed in minor versions of Spark, or be adopted as * first-class Spark API's. + * + * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first + * line of the comment must be ":: Experimental ::" with no trailing blank line. This is because + * of the known issue that Scaladoc displays only either the annotation or the comment, whichever + * comes first. */ @Retention(RetentionPolicy.RUNTIME) @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, 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 891efccf23b6a..5d2ed2b402cd8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -94,26 +94,20 @@ abstract class RDD[T: ClassTag]( def compute(split: Partition, context: TaskContext): Iterator[T] /** - * :: DeveloperApi :: * Implemented by subclasses to return the set of partitions in this RDD. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ - @DeveloperApi protected def getPartitions: Array[Partition] /** - * :: DeveloperApi :: * Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ - @DeveloperApi protected def getDependencies: Seq[Dependency[_]] = deps /** - * :: DeveloperApi :: * Optionally overridden by subclasses to specify placement preferences. */ - @DeveloperApi protected def getPreferredLocations(split: Partition): Seq[String] = Nil /** Optionally overridden by subclasses to specify how they are partitioned. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala index b85eabd6bbdbc..1ce83485f024b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -24,8 +24,12 @@ import org.apache.spark.annotation.DeveloperApi // information about a specific split instance : handles both split instances. // So that we do not need to worry about the differences. @DeveloperApi -class SplitInfo(val inputFormatClazz: Class[_], val hostLocation: String, val path: String, - val length: Long, val underlyingSplit: Any) { +class SplitInfo( + val inputFormatClazz: Class[_], + val hostLocation: String, + val path: String, + val length: Long, + val underlyingSplit: Any) { override def toString(): String = { "SplitInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz + ", hostLocation : " + hostLocation + ", path : " + path + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 90cf8525df523..dee9ef07e41ed 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -391,9 +391,9 @@ object KMeans { * Returns the squared Euclidean distance between two vectors computed by * [[org.apache.spark.mllib.util.MLUtils#fastSquaredDistance]]. */ - private[clustering] - def fastSquaredDistance(v1: BreezeVectorWithNorm, v2: BreezeVectorWithNorm) - : Double = { + private[clustering] def fastSquaredDistance( + v1: BreezeVectorWithNorm, + v2: BreezeVectorWithNorm): Double = { MLUtils.fastSquaredDistance(v1.vector, v1.norm, v2.vector, v2.norm) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala index 027305aa78926..b27e158b43f9a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala @@ -21,9 +21,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.annotation.Experimental -/** - * :: Experimental :: - */ @Experimental trait RegressionModel extends Serializable { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 4d216b5cd14cb..e25201a6c1775 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -249,11 +249,9 @@ class SQLContext(@transient val sparkContext: SparkContext) } /** - * :: DeveloperApi :: * The primary workflow for executing relational queries using Spark. Designed to allow easy * access to the intermediate phases of query execution for developers. */ - @DeveloperApi protected abstract class QueryExecution { def logical: LogicalPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index f2ae5b0fe612f..6cb0e0f61f2d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import net.razorvine.pickle.Pickler import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} -import org.apache.spark.annotation.{AlphaComponent, Experimental, DeveloperApi} +import org.apache.spark.annotation.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index c0d8adf43dd07..b21f24dad785d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -239,7 +239,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { sparkContext.parallelize(Seq(new GenericRow(Array[Any]()): Row), 1) /** Extends QueryExecution with hive specific features. */ - @DeveloperApi protected[sql] abstract class QueryExecution extends super.QueryExecution { // TODO: Create mixin for the analyzer instead of overriding things here. override lazy val optimizedPlan = From ac164b79d12f5a269d9c05d04dced51a415e3b0e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 21 Apr 2014 23:10:53 -0700 Subject: [PATCH 124/641] [SPARK-1459] Use local path (and not complete URL) when opening local lo... ...g file. Author: Marcelo Vanzin Closes #375 from vanzin/event-file and squashes the following commits: f673029 [Marcelo Vanzin] [SPARK-1459] Use local path (and not complete URL) when opening local log file. --- core/src/main/scala/org/apache/spark/util/FileLogger.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 68a12e8ed67d7..7d58d1c765180 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -91,7 +91,7 @@ private[spark] class FileLogger( val dstream = uri.getScheme match { case "file" | null => // Second parameter is whether to append - new FileOutputStream(logPath, !overwrite) + new FileOutputStream(uri.getPath, !overwrite) case _ => val path = new Path(logPath) From 83084d3b7bab09524dd85054c442abaf97d22fea Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 22 Apr 2014 00:42:16 -0700 Subject: [PATCH 125/641] SPARK-1496: Have jarOfClass return Option[String] A simple change, mostly had to change a bunch of example code. Author: Patrick Wendell Closes #438 from pwendell/jar-of-class and squashes the following commits: aa010ff [Patrick Wendell] SPARK-1496: Have jarOfClass return Option[String] --- .../src/main/scala/org/apache/spark/SparkContext.scala | 10 +++++----- .../org/apache/spark/examples/BroadcastTest.scala | 2 +- .../org/apache/spark/examples/CassandraCQLTest.scala | 2 +- .../apache/spark/examples/ExceptionHandlingTest.scala | 2 +- .../scala/org/apache/spark/examples/GroupByTest.scala | 2 +- .../scala/org/apache/spark/examples/HBaseTest.scala | 2 +- .../scala/org/apache/spark/examples/HdfsTest.scala | 2 +- .../scala/org/apache/spark/examples/LogQuery.scala | 2 +- .../org/apache/spark/examples/MultiBroadcastTest.scala | 2 +- .../spark/examples/SimpleSkewedGroupByTest.scala | 2 +- .../org/apache/spark/examples/SkewedGroupByTest.scala | 2 +- .../scala/org/apache/spark/examples/SparkALS.scala | 2 +- .../scala/org/apache/spark/examples/SparkHdfsLR.scala | 2 +- .../scala/org/apache/spark/examples/SparkKMeans.scala | 2 +- .../main/scala/org/apache/spark/examples/SparkLR.scala | 2 +- .../org/apache/spark/examples/SparkPageRank.scala | 2 +- .../main/scala/org/apache/spark/examples/SparkPi.scala | 2 +- .../main/scala/org/apache/spark/examples/SparkTC.scala | 2 +- .../org/apache/spark/examples/SparkTachyonHdfsLR.scala | 2 +- .../org/apache/spark/examples/SparkTachyonPi.scala | 2 +- .../apache/spark/examples/mllib/TallSkinnyPCA.scala | 2 +- .../apache/spark/examples/mllib/TallSkinnySVD.scala | 2 +- .../spark/streaming/examples/ActorWordCount.scala | 2 +- .../spark/streaming/examples/FlumeEventCount.scala | 2 +- .../spark/streaming/examples/HdfsWordCount.scala | 2 +- .../spark/streaming/examples/KafkaWordCount.scala | 2 +- .../spark/streaming/examples/MQTTWordCount.scala | 2 +- .../spark/streaming/examples/NetworkWordCount.scala | 2 +- .../apache/spark/streaming/examples/QueueStream.scala | 2 +- .../spark/streaming/examples/RawNetworkGrep.scala | 2 +- .../examples/RecoverableNetworkWordCount.scala | 2 +- .../streaming/examples/StatefulNetworkWordCount.scala | 2 +- .../spark/streaming/examples/TwitterAlgebirdCMS.scala | 2 +- .../spark/streaming/examples/TwitterAlgebirdHLL.scala | 2 +- .../spark/streaming/examples/TwitterPopularTags.scala | 2 +- .../spark/streaming/examples/ZeroMQWordCount.scala | 2 +- .../examples/clickstream/PageViewStream.scala | 2 +- .../org/apache/spark/streaming/StreamingContext.scala | 2 +- 38 files changed, 42 insertions(+), 42 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index aa49f32ecdf62..25ca650a3a37e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1346,19 +1346,19 @@ object SparkContext extends Logging { * Find the JAR from which a given class was loaded, to make it easy for users to pass * their JARs to SparkContext. */ - def jarOfClass(cls: Class[_]): Seq[String] = { + def jarOfClass(cls: Class[_]): Option[String] = { val uri = cls.getResource("/" + cls.getName.replace('.', '/') + ".class") if (uri != null) { val uriStr = uri.toString if (uriStr.startsWith("jar:file:")) { // URI will be of the form "jar:file:/path/foo.jar!/package/cls.class", // so pull out the /path/foo.jar - List(uriStr.substring("jar:file:".length, uriStr.indexOf('!'))) + Some(uriStr.substring("jar:file:".length, uriStr.indexOf('!'))) } else { - Nil + None } } else { - Nil + None } } @@ -1367,7 +1367,7 @@ object SparkContext extends Logging { * to pass their JARs to SparkContext. In most cases you can call jarOfObject(this) in * your driver program. */ - def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass) + def jarOfObject(obj: AnyRef): Option[String] = jarOfClass(obj.getClass) /** * Creates a modified version of a SparkConf with the parameters that can be passed separately diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index c8c916bb45e00..f6dfd2c4c6217 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -35,7 +35,7 @@ object BroadcastTest { System.setProperty("spark.broadcast.blockSize", blockSize) val sc = new SparkContext(args(0), "Broadcast Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val slices = if (args.length > 1) args(1).toInt else 2 val num = if (args.length > 2) args(2).toInt else 1000000 diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala index 4e787240e912d..3798329fc2f41 100644 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala @@ -75,7 +75,7 @@ object CassandraCQLTest { val sc = new SparkContext(args(0), "CQLTestApp", System.getenv("SPARK_HOME"), - SparkContext.jarOfClass(this.getClass)) + SparkContext.jarOfClass(this.getClass).toSeq) val cHost: String = args(1) val cPort: String = args(2) val KeySpace = "retail" diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala index be7d39549a28d..f0dcef431b2e1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala @@ -27,7 +27,7 @@ object ExceptionHandlingTest { } val sc = new SparkContext(args(0), "ExceptionHandlingTest", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) sc.parallelize(0 until sc.defaultParallelism).foreach { i => if (math.random > 0.75) { throw new Exception("Testing exception handling") diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index 2b7ecdc991325..e67bb29a49405 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -36,7 +36,7 @@ object GroupByTest { var numReducers = if (args.length > 4) args(4).toInt else numMappers val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala index cbf78e8e9eba1..adbd1c02fa2ea 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala @@ -27,7 +27,7 @@ import org.apache.spark.rdd.NewHadoopRDD object HBaseTest { def main(args: Array[String]) { val sc = new SparkContext(args(0), "HBaseTest", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val conf = HBaseConfiguration.create() diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index dd6d5205133be..c7a4884af10b7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -22,7 +22,7 @@ import org.apache.spark._ object HdfsTest { def main(args: Array[String]) { val sc = new SparkContext(args(0), "HdfsTest", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val file = sc.textFile(args(1)) val mapped = file.map(s => s.length).cache() for (iter <- 1 to 10) { diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index 35758fa003d94..820e87d04f3fa 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -46,7 +46,7 @@ object LogQuery { } val sc = new SparkContext(args(0), "Log Query", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val dataSet = if (args.length == 2) sc.textFile(args(1)) diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index 58f26f1e24052..c8985eae33de3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -28,7 +28,7 @@ object MultiBroadcastTest { } val sc = new SparkContext(args(0), "Multi-Broadcast Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val slices = if (args.length > 1) args(1).toInt else 2 val num = if (args.length > 2) args(2).toInt else 1000000 diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 557a0c1841339..54e8503711e30 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -37,7 +37,7 @@ object SimpleSkewedGroupByTest { var ratio = if (args.length > 5) args(5).toInt else 5.0 val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 05a74725b875b..1c5f22e1c00bb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -36,7 +36,7 @@ object SkewedGroupByTest { var numReducers = if (args.length > 4) args(4).toInt else numMappers val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random 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 191c82fd913ee..0dc726aecdd28 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -113,7 +113,7 @@ object SparkALS { printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) val sc = new SparkContext(host, "SparkALS", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val R = generateR() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index fd63ba3dbce7d..3a6f18c33ea4b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -56,7 +56,7 @@ object SparkHdfsLR { val inputPath = args(1) val conf = SparkHadoopUtil.get.newConfiguration() val sc = new SparkContext(args(0), "SparkHdfsLR", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq, Map(), InputFormatInfo.computePreferredLocations( Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) )) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index 8aa31d7e6a2c2..dcae9591b0407 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -57,7 +57,7 @@ object SparkKMeans { System.exit(1) } val sc = new SparkContext(args(0), "SparkLocalKMeans", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val lines = sc.textFile(args(1)) val data = lines.map(parseVector _).cache() val K = args(2).toInt diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index d70ce603bb71d..4f74882ccbea5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -52,7 +52,7 @@ object SparkLR { System.exit(1) } val sc = new SparkContext(args(0), "SparkLR", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val numSlices = if (args.length > 1) args(1).toInt else 2 val points = sc.parallelize(generateData, numSlices).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index 60e4a11a21f69..fa41c5c560943 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -37,7 +37,7 @@ object SparkPageRank { } var iters = args(2).toInt val ctx = new SparkContext(args(0), "PageRank", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val lines = ctx.textFile(args(1), 1) val links = lines.map{ s => val parts = s.split("\\s+") diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index d3babc3ed12c8..d8f5720504223 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -29,7 +29,7 @@ object SparkPi { System.exit(1) } val spark = new SparkContext(args(0), "SparkPi", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val slices = if (args.length > 1) args(1).toInt else 2 val n = 100000 * slices val count = spark.parallelize(1 to n, slices).map { i => diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index 65bd61abda6cd..17d983cd875db 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -47,7 +47,7 @@ object SparkTC { System.exit(1) } val spark = new SparkContext(args(0), "SparkTC", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val slices = if (args.length > 1) args(1).toInt else 2 var tc = spark.parallelize(generateGraph, slices).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala index 4f558929add51..7e43c384bdb9d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala @@ -58,7 +58,7 @@ object SparkTachyonHdfsLR { val inputPath = args(1) val conf = SparkHadoopUtil.get.newConfiguration() val sc = new SparkContext(args(0), "SparkTachyonHdfsLR", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq, Map(), InputFormatInfo.computePreferredLocations( Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) )) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala index 2b207fd8d3e16..93459110e4e0e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala @@ -33,7 +33,7 @@ object SparkTachyonPi { System.exit(1) } val spark = new SparkContext(args(0), "SparkTachyonPi", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val slices = if (args.length > 1) args(1).toInt else 2 val n = 100000 * slices diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala index 61b9655cd3759..39e71cdab4328 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -44,7 +44,7 @@ object TallSkinnyPCA { .setMaster(args(0)) .setAppName("TallSkinnyPCA") .setSparkHome(System.getenv("SPARK_HOME")) - .setJars(SparkContext.jarOfClass(this.getClass)) + .setJars(SparkContext.jarOfClass(this.getClass).toSeq) val sc = new SparkContext(conf) // Load and parse the data file. diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala index 9aeebf58eabfb..2b7de2acc65d6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -44,7 +44,7 @@ object TallSkinnySVD { .setMaster(args(0)) .setAppName("TallSkinnySVD") .setSparkHome(System.getenv("SPARK_HOME")) - .setJars(SparkContext.jarOfClass(this.getClass)) + .setJars(SparkContext.jarOfClass(this.getClass).toSeq) val sc = new SparkContext(conf) // Load and parse the data file. diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index eb44768b9c101..c845dd8904c90 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -150,7 +150,7 @@ object ActorWordCount { // Create the context and set the batch size val ssc = new StreamingContext(master, "ActorWordCount", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) /* * Following is the use of actorStream to plug in custom actor as receiver diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala index 11c3aaad3c8a8..26b6024534124 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala @@ -51,7 +51,7 @@ object FlumeEventCount { val batchInterval = Milliseconds(2000) // Create the context and set the batch size val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval, - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create a flume stream val stream = FlumeUtils.createStream(ssc, host,port,StorageLevel.MEMORY_ONLY_SER_2) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala index 1c0ce3111e290..7f86fc792aacf 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala @@ -41,7 +41,7 @@ object HdfsWordCount { // Create the context val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create the FileInputDStream on the directory and use the // stream to count words in new files created diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index cca0be2cbb9c9..3a18bd02b89ac 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -52,7 +52,7 @@ object KafkaWordCount { val Array(master, zkQuorum, group, topics, numThreads) = args val ssc = new StreamingContext(master, "KafkaWordCount", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) ssc.checkpoint("checkpoint") val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index 656222e0c1b31..3d3c27ed78204 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -97,7 +97,7 @@ object MQTTWordCount { val Seq(master, brokerUrl, topic) = args.toSeq val ssc = new StreamingContext(master, "MqttWordCount", Seconds(2), System.getenv("SPARK_HOME"), - StreamingContext.jarOfClass(this.getClass)) + StreamingContext.jarOfClass(this.getClass).toSeq) val lines = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_ONLY_SER_2) val words = lines.flatMap(x => x.toString.split(" ")) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala index d4c4d86b3466c..272ab11212451 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala @@ -47,7 +47,7 @@ object NetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala index 612ecf7b7821a..ff2a205ec1c15 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala @@ -35,7 +35,7 @@ object QueueStream { // Create the context val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create the queue through which RDDs can be pushed to // a QueueInputDStream diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala index cdd7547d0d3b4..d915c0c39b334 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala @@ -50,7 +50,7 @@ object RawNetworkGrep { // Create the context val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Warm up the JVMs on master and slave for JIT compilation to kick in RawTextHelper.warmUp(ssc.sparkContext) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala index 56d10a964b71b..4aacbb1991418 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala @@ -77,7 +77,7 @@ object RecoverableNetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(master, "RecoverableNetworkWordCount", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala index 14f65a2f8d46c..ef94c9298dd93 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -54,7 +54,7 @@ object StatefulNetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", - Seconds(1), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + Seconds(1), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) ssc.checkpoint(".") // Create a NetworkInputDStream on target ip:port and count the diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala index 8a654f8fada21..c38905e8f3663 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -68,7 +68,7 @@ object TwitterAlgebirdCMS { val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER_2) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala index 45771d7050eeb..c067046f9020e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -55,7 +55,7 @@ object TwitterAlgebirdHLL { val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala index 896d010c68f18..2597c8178862a 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala @@ -41,7 +41,7 @@ object TwitterPopularTags { val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) val stream = TwitterUtils.createStream(ssc, None, filters) val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index 6f88db1abf19d..109ff855b5228 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -86,7 +86,7 @@ object ZeroMQWordCount { // Create the context and set the batch size val ssc = new StreamingContext(master, "ZeroMQWordCount", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala index 2b130fb30eaaa..edc769c59734a 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala @@ -45,7 +45,7 @@ object PageViewStream { // Create the context val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create a NetworkInputDStream on target host:port and convert each line to a PageView val pageViews = ssc.socketTextStream(host, port) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index daa5c69bbadbf..6d9dc87a707ac 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -543,7 +543,7 @@ object StreamingContext extends Logging { * Find the JAR from which a given class was loaded, to make it easy for users to pass * their JARs to StreamingContext. */ - def jarOfClass(cls: Class[_]): Seq[String] = SparkContext.jarOfClass(cls) + def jarOfClass(cls: Class[_]): Option[String] = SparkContext.jarOfClass(cls) private[streaming] def createNewSparkContext(conf: SparkConf): SparkContext = { // Set the default cleaner delay to an hour if not already set. From 87de29084eed2a287ee114d492f45b211c500c6f Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 22 Apr 2014 09:43:13 -0700 Subject: [PATCH 126/641] [HOTFIX] SPARK-1399: remove outdated comments as the original PR was merged before this mistake is found....fix here, Sorry about that @pwendell, @andrewor14, I will be more careful next time Author: CodingCat Closes #474 from CodingCat/hotfix_1399 and squashes the following commits: f3a8ba9 [CodingCat] move outdated comments --- core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 2eb8c7a3a3073..153434a2032be 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -37,7 +37,6 @@ private[ui] class StageTableBase( protected def isFairScheduler = parent.isFairScheduler protected def columns: Seq[Node] = { - // create dummy element to wrap the columns Stage Id ++ {if (isFairScheduler) {Pool Name} else Seq.empty} ++ Description From 0f87e6ad4366a8c453a7415bc89399030003c264 Mon Sep 17 00:00:00 2001 From: Ahir Reddy Date: Tue, 22 Apr 2014 09:44:41 -0700 Subject: [PATCH 127/641] [SPARK-1560]: Updated Pyrolite Dependency to be Java 6 compatible Changed the Pyrolite dependency to a build which targets Java 6. Author: Ahir Reddy Closes #479 from ahirreddy/java6-pyrolite and squashes the following commits: 8ea25d3 [Ahir Reddy] Updated maven build to use java 6 compatible pyrolite dabc703 [Ahir Reddy] Updated Pyrolite dependency to be Java 6 compatible --- core/pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index d87e2bca030e3..058b7acba73ca 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -269,7 +269,7 @@ org.spark-project pyrolite - 2.0 + 2.0.1 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f115f0d3a041c..1ca4e0dc8add1 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -352,7 +352,7 @@ object SparkBuild extends Build { "com.twitter" % "chill-java" % chillVersion excludeAll(excludeAsm), "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), "com.clearspring.analytics" % "stream" % "2.5.1" excludeAll(excludeFastutil), - "org.spark-project" % "pyrolite" % "2.0" + "org.spark-project" % "pyrolite" % "2.0.1" ), libraryDependencies ++= maybeAvro ) From c919798f0912dc03c8365b9a384d9ee6d5b25c51 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Tue, 22 Apr 2014 11:06:18 -0700 Subject: [PATCH 128/641] fix bugs of dot in python If there are no `transpose()` in `self.theta`, a *ValueError: matrices are not aligned* is occurring. The former test case just ignore this situation. Author: Xusen Yin Closes #463 from yinxusen/python-naive-bayes and squashes the following commits: fcbe3bc [Xusen Yin] fix bugs of dot in python --- python/pyspark/mllib/classification.py | 2 +- python/pyspark/mllib/tests.py | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 3a23e0801fe7b..c5844597c95f2 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -154,7 +154,7 @@ def __init__(self, labels, pi, theta): def predict(self, x): """Return the most likely class for a data vector x""" - return self.labels[numpy.argmax(self.pi + _dot(x, self.theta))] + return self.labels[numpy.argmax(self.pi + _dot(x, self.theta.transpose()))] class NaiveBayes(object): @classmethod diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index d4771d779f9f4..1ee96bb4af37b 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -104,10 +104,10 @@ def test_clustering(self): def test_classification(self): from pyspark.mllib.classification import LogisticRegressionWithSGD, SVMWithSGD, NaiveBayes data = [ - LabeledPoint(0.0, [1, 0]), - LabeledPoint(1.0, [0, 1]), - LabeledPoint(0.0, [2, 0]), - LabeledPoint(1.0, [0, 2]) + LabeledPoint(0.0, [1, 0, 0]), + LabeledPoint(1.0, [0, 1, 1]), + LabeledPoint(0.0, [2, 0, 0]), + LabeledPoint(1.0, [0, 2, 1]) ] rdd = self.sc.parallelize(data) features = [p.features.tolist() for p in data] From bf9d49b6d1f668b49795c2d380ab7d64ec0029da Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Tue, 22 Apr 2014 11:07:30 -0700 Subject: [PATCH 129/641] [SPARK-1281] Improve partitioning in ALS ALS was using HashPartitioner and explicit uses of `%` together. Further, the naked use of `%` meant that, if the number of partitions corresponded with the stride of arithmetic progressions appearing in user and product ids, users and products could be mapped into buckets in an unfair or unwise way. This pull request: 1) Makes the Partitioner an instance variable of ALS. 2) Replaces the direct uses of `%` with calls to a Partitioner. 3) Defines an anonymous Partitioner that scrambles the bits of the object's hashCode before reducing to the number of present buckets. This pull request does not make the partitioner user-configurable. I'm not all that happy about the way I did (1). It introduces an icky lifetime issue and dances around it by nulling something. However, I don't know a better way to make the partitioner visible everywhere it needs to be visible. Author: Tor Myklebust Closes #407 from tmyklebu/master and squashes the following commits: dcf583a [Tor Myklebust] Remove the partitioner member variable; instead, thread that needle everywhere it needs to go. 23d6f91 [Tor Myklebust] Stop making the partitioner configurable. 495784f [Tor Myklebust] Merge branch 'master' of https://github.com/apache/spark 674933a [Tor Myklebust] Fix style. 40edc23 [Tor Myklebust] Fix missing space. f841345 [Tor Myklebust] Fix daft bug creating 'pairs', also for -> foreach. 5ec9e6c [Tor Myklebust] Clean a couple of things up using 'map'. 36a0f43 [Tor Myklebust] Make the partitioner private. d872b09 [Tor Myklebust] Add negative id ALS test. df27697 [Tor Myklebust] Support custom partitioners. Currently we use the same partitioner for users and products. c90b6d8 [Tor Myklebust] Scramble user and product ids before bucketing. c774d7d [Tor Myklebust] Make the partitioner a member variable and use it instead of modding directly. --- .../spark/mllib/recommendation/ALS.scala | 47 +++++++++++-------- .../spark/mllib/recommendation/ALSSuite.scala | 30 ++++++++++-- 2 files changed, 54 insertions(+), 23 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 1f5c746a3457c..60fb73f2b5be5 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 @@ -21,6 +21,7 @@ import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.math.{abs, sqrt} import scala.util.Random import scala.util.Sorting +import scala.util.hashing.byteswap32 import com.esotericsoftware.kryo.Kryo import org.jblas.{DoubleMatrix, SimpleBlas, Solve} @@ -32,6 +33,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.rdd.RDD import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.SparkContext._ +import org.apache.spark.util.Utils /** * Out-link information for a user or product block. This includes the original user/product IDs @@ -169,34 +171,39 @@ class ALS private ( this.numBlocks } - val partitioner = new HashPartitioner(numBlocks) + val partitioner = new Partitioner { + val numPartitions = numBlocks - val ratingsByUserBlock = ratings.map{ rating => (rating.user % numBlocks, rating) } + def getPartition(x: Any): Int = { + Utils.nonNegativeMod(byteswap32(x.asInstanceOf[Int]), numPartitions) + } + } + + val ratingsByUserBlock = ratings.map{ rating => + (partitioner.getPartition(rating.user), rating) + } val ratingsByProductBlock = ratings.map{ rating => - (rating.product % numBlocks, Rating(rating.product, rating.user, rating.rating)) + (partitioner.getPartition(rating.product), + Rating(rating.product, rating.user, rating.rating)) } - val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) - val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) + val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock, partitioner) + val (productInLinks, productOutLinks) = + makeLinkRDDs(numBlocks, ratingsByProductBlock, partitioner) // Initialize user and product factors randomly, but use a deterministic seed for each // partition so that fault recovery works 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 - def hash(x: Int): Int = { - val r = x ^ (x >>> 20) ^ (x >>> 12) - r ^ (r >>> 7) ^ (r >>> 4) - } var users = userOutLinks.mapPartitionsWithIndex { (index, itr) => - val rand = new Random(hash(seed1 ^ index)) + val rand = new Random(byteswap32(seed1 ^ index)) itr.map { case (x, y) => (x, y.elementIds.map(_ => randomFactor(rank, rand))) } } var products = productOutLinks.mapPartitionsWithIndex { (index, itr) => - val rand = new Random(hash(seed2 ^ index)) + val rand = new Random(byteswap32(seed2 ^ index)) itr.map { case (x, y) => (x, y.elementIds.map(_ => randomFactor(rank, rand))) } @@ -327,13 +334,14 @@ class ALS private ( * Make the out-links table for a block of the users (or products) dataset given the list of * (user, product, rating) values for the users in that block (or the opposite for products). */ - private def makeOutLinkBlock(numBlocks: Int, ratings: Array[Rating]): OutLinkBlock = { + private def makeOutLinkBlock(numBlocks: Int, ratings: Array[Rating], + partitioner: Partitioner): OutLinkBlock = { val userIds = ratings.map(_.user).distinct.sorted val numUsers = userIds.length val userIdToPos = userIds.zipWithIndex.toMap val shouldSend = Array.fill(numUsers)(new BitSet(numBlocks)) for (r <- ratings) { - shouldSend(userIdToPos(r.user))(r.product % numBlocks) = true + shouldSend(userIdToPos(r.user))(partitioner.getPartition(r.product)) = true } OutLinkBlock(userIds, shouldSend) } @@ -342,14 +350,15 @@ class ALS private ( * Make the in-links table for a block of the users (or products) dataset given a list of * (user, product, rating) values for the users in that block (or the opposite for products). */ - private def makeInLinkBlock(numBlocks: Int, ratings: Array[Rating]): InLinkBlock = { + private def makeInLinkBlock(numBlocks: Int, ratings: Array[Rating], + partitioner: Partitioner): InLinkBlock = { val userIds = ratings.map(_.user).distinct.sorted val numUsers = userIds.length val userIdToPos = userIds.zipWithIndex.toMap // Split out our ratings by product block val blockRatings = Array.fill(numBlocks)(new ArrayBuffer[Rating]) for (r <- ratings) { - blockRatings(r.product % numBlocks) += r + blockRatings(partitioner.getPartition(r.product)) += r } val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numBlocks) for (productBlock <- 0 until numBlocks) { @@ -374,14 +383,14 @@ class ALS private ( * the users (or (blockId, (p, u, r)) for the products). We create these simultaneously to avoid * having to shuffle the (blockId, (u, p, r)) RDD twice, or to cache it. */ - private def makeLinkRDDs(numBlocks: Int, ratings: RDD[(Int, Rating)]) + private def makeLinkRDDs(numBlocks: Int, ratings: RDD[(Int, Rating)], partitioner: Partitioner) : (RDD[(Int, InLinkBlock)], RDD[(Int, OutLinkBlock)]) = { val grouped = ratings.partitionBy(new HashPartitioner(numBlocks)) val links = grouped.mapPartitionsWithIndex((blockId, elements) => { val ratings = elements.map{_._2}.toArray - val inLinkBlock = makeInLinkBlock(numBlocks, ratings) - val outLinkBlock = makeOutLinkBlock(numBlocks, ratings) + val inLinkBlock = makeInLinkBlock(numBlocks, ratings, partitioner) + val outLinkBlock = makeOutLinkBlock(numBlocks, ratings, partitioner) Iterator.single((blockId, (inLinkBlock, outLinkBlock))) }, true) val inLinks = links.mapValues(_._1) 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 5aab9aba8f9c0..4dfcd4b52ec66 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 @@ -27,6 +27,7 @@ import org.jblas.DoubleMatrix import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.Partitioner object ALSSuite { @@ -74,7 +75,6 @@ object ALSSuite { (sampledRatings, trueRatings, truePrefs) } - } @@ -128,6 +128,25 @@ class ALSSuite extends FunSuite with LocalSparkContext { assert(u11 != u2) } + test("negative ids") { + val data = ALSSuite.generateRatings(50, 50, 2, 0.7, false, false) + val ratings = sc.parallelize(data._1.map { case Rating(u, p, r) => + Rating(u - 25, p - 25, r) + }) + val correct = data._2 + val model = ALS.train(ratings, 5, 15) + + val pairs = Array.tabulate(50, 50)((u, p) => (u - 25, p - 25)).flatten + val ans = model.predict(sc.parallelize(pairs)).collect() + ans.foreach { r => + val u = r.user + 25 + val p = r.product + 25 + val v = r.rating + val error = v - correct.get(u, p) + assert(math.abs(error) < 0.4) + } + } + /** * Test if we can correctly factorize R = U * P where U and P are of known rank. * @@ -140,16 +159,19 @@ class ALSSuite extends FunSuite with LocalSparkContext { * @param implicitPrefs flag to test implicit feedback * @param bulkPredict flag to test bulk prediciton * @param negativeWeights whether the generated data can contain negative values + * @param numBlocks number of blocks to partition users and products into */ def testALS(users: Int, products: Int, features: Int, iterations: Int, samplingRate: Double, matchThreshold: Double, implicitPrefs: Boolean = false, - bulkPredict: Boolean = false, negativeWeights: Boolean = false) + bulkPredict: Boolean = false, negativeWeights: Boolean = false, numBlocks: Int = -1) { val (sampledRatings, trueRatings, truePrefs) = ALSSuite.generateRatings(users, products, features, samplingRate, implicitPrefs, negativeWeights) val model = implicitPrefs match { - case false => ALS.train(sc.parallelize(sampledRatings), features, iterations) - case true => ALS.trainImplicit(sc.parallelize(sampledRatings), features, iterations) + case false => ALS.train(sc.parallelize(sampledRatings), features, iterations, 0.01, + numBlocks, 0L) + case true => ALS.trainImplicit(sc.parallelize(sampledRatings), features, iterations, 0.01, + numBlocks, 1.0, 0L) } val predictedU = new DoubleMatrix(users, features) From 26d35f3fd942761b0adecd1a720e1fa834db4de9 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 22 Apr 2014 11:20:47 -0700 Subject: [PATCH 130/641] [SPARK-1506][MLLIB] Documentation improvements for MLlib 1.0 Preview: http://54.82.240.23:4000/mllib-guide.html Table of contents: * Basics * Data types * Summary statistics * Classification and regression * linear support vector machine (SVM) * logistic regression * linear linear squares, Lasso, and ridge regression * decision tree * naive Bayes * Collaborative Filtering * alternating least squares (ALS) * Clustering * k-means * Dimensionality reduction * singular value decomposition (SVD) * principal component analysis (PCA) * Optimization * stochastic gradient descent * limited-memory BFGS (L-BFGS) Author: Xiangrui Meng Closes #422 from mengxr/mllib-doc and squashes the following commits: 944e3a9 [Xiangrui Meng] merge master f9fda28 [Xiangrui Meng] minor 9474065 [Xiangrui Meng] add alpha to ALS examples 928e630 [Xiangrui Meng] initialization_mode -> initializationMode 5bbff49 [Xiangrui Meng] add imports to labeled point examples c17440d [Xiangrui Meng] fix python nb example 28f40dc [Xiangrui Meng] remove localhost:4000 369a4d3 [Xiangrui Meng] Merge branch 'master' into mllib-doc 7dc95cc [Xiangrui Meng] update linear methods 053ad8a [Xiangrui Meng] add links to go back to the main page abbbf7e [Xiangrui Meng] update ALS argument names 648283e [Xiangrui Meng] level down statistics 14e2287 [Xiangrui Meng] add sample libsvm data and use it in guide 8cd2441 [Xiangrui Meng] minor updates 186ab07 [Xiangrui Meng] update section names 6568d65 [Xiangrui Meng] update toc, level up lr and svm 162ee12 [Xiangrui Meng] rename section names 5c1e1b1 [Xiangrui Meng] minor 8aeaba1 [Xiangrui Meng] wrap long lines 6ce6a6f [Xiangrui Meng] add summary statistics to toc 5760045 [Xiangrui Meng] claim beta cc604bf [Xiangrui Meng] remove classification and regression 92747b3 [Xiangrui Meng] make section titles consistent e605dd6 [Xiangrui Meng] add LIBSVM loader f639674 [Xiangrui Meng] add python section to migration guide c82ffb4 [Xiangrui Meng] clean optimization 31660eb [Xiangrui Meng] update linear algebra and stat 0a40837 [Xiangrui Meng] first pass over linear methods 1fc8271 [Xiangrui Meng] update toc 906ed0a [Xiangrui Meng] add a python example to naive bayes 5f0a700 [Xiangrui Meng] update collaborative filtering 656d416 [Xiangrui Meng] update mllib-clustering 86e143a [Xiangrui Meng] remove data types section from main page 8d1a128 [Xiangrui Meng] move part of linear algebra to data types and add Java/Python examples d1b5cbf [Xiangrui Meng] merge master 72e4804 [Xiangrui Meng] one pass over tree guide 64f8995 [Xiangrui Meng] move decision tree guide to a separate file 9fca001 [Xiangrui Meng] add first version of linear algebra guide 53c9552 [Xiangrui Meng] update dependencies f316ec2 [Xiangrui Meng] add migration guide f399f6c [Xiangrui Meng] move linear-algebra to dimensionality-reduction 182460f [Xiangrui Meng] add guide for naive Bayes 137fd1d [Xiangrui Meng] re-organize toc a61e434 [Xiangrui Meng] update mllib's toc --- docs/mllib-basics.md | 476 ++++++++++++++++++++ docs/mllib-classification-regression.md | 568 ------------------------ docs/mllib-clustering.md | 44 +- docs/mllib-collaborative-filtering.md | 78 ++-- docs/mllib-decision-tree.md | 185 ++++++++ docs/mllib-dimensionality-reduction.md | 86 ++++ docs/mllib-guide.md | 172 ++++--- docs/mllib-linear-algebra.md | 74 --- docs/mllib-linear-methods.md | 389 ++++++++++++++++ docs/mllib-naive-bayes.md | 115 +++++ docs/mllib-optimization.md | 25 +- mllib/data/sample_libsvm_data.txt | 100 +++++ 12 files changed, 1543 insertions(+), 769 deletions(-) create mode 100644 docs/mllib-basics.md delete mode 100644 docs/mllib-classification-regression.md create mode 100644 docs/mllib-decision-tree.md create mode 100644 docs/mllib-dimensionality-reduction.md delete mode 100644 docs/mllib-linear-algebra.md create mode 100644 docs/mllib-linear-methods.md create mode 100644 docs/mllib-naive-bayes.md create mode 100644 mllib/data/sample_libsvm_data.txt diff --git a/docs/mllib-basics.md b/docs/mllib-basics.md new file mode 100644 index 0000000000000..710ce1721fe25 --- /dev/null +++ b/docs/mllib-basics.md @@ -0,0 +1,476 @@ +--- +layout: global +title: MLlib - Basics +--- + +* Table of contents +{:toc} + +MLlib supports local vectors and matrices stored on a single machine, +as well as distributed matrices backed by one or more RDDs. +In the current implementation, local vectors and matrices are simple data models +to serve public interfaces. The underly linear algebra operations are provided by +[Breeze](http://www.scalanlp.org/) and [jblas](http://jblas.org/). +A training example used in supervised learning is called "labeled point" in MLlib. + +## Local vector + +A local vector has integer-typed and 0-based indices and double-typed values, stored on a single +machine. MLlib supports two types of local vectors: dense and sparse. A dense vector is backed by +a double array representing its entry values, while a sparse vector is backed by two parallel +arrays: indices and values. For example, a vector $(1.0, 0.0, 3.0)$ can be represented in dense +format as `[1.0, 0.0, 3.0]` or in sparse format as `(3, [0, 2], [1.0, 3.0])`, where `3` is the size +of the vector. + +
      +
      + +The base class of local vectors is +[`Vector`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector), and we provide two +implementations: [`DenseVector`](api/mllib/index.html#org.apache.spark.mllib.linalg.DenseVector) and +[`SparseVector`](api/mllib/index.html#org.apache.spark.mllib.linalg.SparseVector). We recommend +using the factory methods implemented in +[`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) to create local vectors. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.{Vector, Vectors} + +// Create a dense vector (1.0, 0.0, 3.0). +val dv: Vector = Vectors.dense(1.0, 0.0, 3.0) +// Create a sparse vector (1.0, 0.0, 3.0) by specifying its indices and values corresponding to nonzero entries. +val sv1: Vector = Vectors.sparse(3, Array(0, 2), Array(1.0, 3.0)) +// Create a sparse vector (1.0, 0.0, 3.0) by specifying its nonzero entries. +val sv2: Vector = Vectors.sparse(3, Seq((0, 1.0), (2, 3.0))) +{% endhighlight %} + +***Note*** + +Scala imports `scala.collection.immutable.Vector` by default, so you have to import +`org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`. + +
      + +
      + +The base class of local vectors is +[`Vector`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector), and we provide two +implementations: [`DenseVector`](api/mllib/index.html#org.apache.spark.mllib.linalg.DenseVector) and +[`SparseVector`](api/mllib/index.html#org.apache.spark.mllib.linalg.SparseVector). We recommend +using the factory methods implemented in +[`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) to create local vectors. + +{% highlight java %} +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; + +// Create a dense vector (1.0, 0.0, 3.0). +Vector dv = Vectors.dense(1.0, 0.0, 3.0); +// Create a sparse vector (1.0, 0.0, 3.0) by specifying its indices and values corresponding to nonzero entries. +Vector sv = Vectors.sparse(3, new int[] {0, 2}, new double[] {1.0, 3.0}); +{% endhighlight %} +
      + +
      +MLlib recognizes the following types as dense vectors: + +* NumPy's [`array`](http://docs.scipy.org/doc/numpy/reference/generated/numpy.array.html) +* Python's list, e.g., `[1, 2, 3]` + +and the following as sparse vectors: + +* MLlib's [`SparseVector`](api/pyspark/pyspark.mllib.linalg.SparseVector-class.html). +* SciPy's + [`csc_matrix`](http://docs.scipy.org/doc/scipy/reference/generated/scipy.sparse.csc_matrix.html#scipy.sparse.csc_matrix) + with a single column + +We recommend using NumPy arrays over lists for efficiency, and using the factory methods implemented +in [`Vectors`](api/pyspark/pyspark.mllib.linalg.Vectors-class.html) to create sparse vectors. + +{% highlight python %} +import numpy as np +import scipy.sparse as sps +from pyspark.mllib.linalg import Vectors + +# Use a NumPy array as a dense vector. +dv1 = np.array([1.0, 0.0, 3.0]) +# Use a Python list as a dense vector. +dv2 = [1.0, 0.0, 3.0] +# Create a SparseVector. +sv1 = Vectors.sparse(3, [0, 2], [1.0, 3.0]) +# Use a single-column SciPy csc_matrix as a sparse vector. +sv2 = sps.csc_matrix((np.array([1.0, 3.0]), np.array([0, 2]), np.array([0, 2])), shape = (3, 1)) +{% endhighlight %} + +
      +
      + +## Labeled point + +A labeled point is a local vector, either dense or sparse, associated with a label/response. +In MLlib, labeled points are used in supervised learning algorithms. +We use a double to store a label, so we can use labeled points in both regression and classification. +For binary classification, label should be either $0$ (negative) or $1$ (positive). +For multiclass classification, labels should be class indices staring from zero: $0, 1, 2, \ldots$. + +
      + +
      + +A labeled point is represented by the case class +[`LabeledPoint`](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint). + +{% highlight scala %} +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint + +// Create a labeled point with a positive label and a dense feature vector. +val pos = LabeledPoint(1.0, Vectors.dense(1.0, 0.0, 3.0)) + +// Create a labeled point with a negative label and a sparse feature vector. +val neg = LabeledPoint(0.0, Vectors.sparse(3, Array(0, 2), Array(1.0, 3.0))) +{% endhighlight %} +
      + +
      + +A labeled point is represented by +[`LabeledPoint`](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint). + +{% highlight java %} +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; + +// Create a labeled point with a positive label and a dense feature vector. +LabeledPoint pos = new LabeledPoint(1.0, Vectors.dense(1.0, 0.0, 3.0)); + +// Create a labeled point with a negative label and a sparse feature vector. +LabeledPoint neg = new LabeledPoint(1.0, Vectors.sparse(3, new int[] {0, 2}, new double[] {1.0, 3.0})); +{% endhighlight %} +
      + +
      + +A labeled point is represented by +[`LabeledPoint`](api/pyspark/pyspark.mllib.regression.LabeledPoint-class.html). + +{% highlight python %} +from pyspark.mllib.linalg import SparseVector +from pyspark.mllib.regression import LabeledPoint + +# Create a labeled point with a positive label and a dense feature vector. +pos = LabeledPoint(1.0, [1.0, 0.0, 3.0]) + +# Create a labeled point with a negative label and a sparse feature vector. +neg = LabeledPoint(0.0, SparseVector(3, [0, 2], [1.0, 3.0])) +{% endhighlight %} +
      +
      + +***Sparse data*** + +It is very common in practice to have sparse training data. MLlib supports reading training +examples stored in `LIBSVM` format, which is the default format used by +[`LIBSVM`](http://www.csie.ntu.edu.tw/~cjlin/libsvm/) and +[`LIBLINEAR`](http://www.csie.ntu.edu.tw/~cjlin/liblinear/). It is a text format. Each line +represents a labeled sparse feature vector using the following format: + +~~~ +label index1:value1 index2:value2 ... +~~~ + +where the indices are one-based and in ascending order. +After loading, the feature indices are converted to zero-based. + +
      +
      + +[`MLUtils.loadLibSVMData`](api/mllib/index.html#org.apache.spark.mllib.util.MLUtils$) reads training +examples stored in LIBSVM format. + +{% highlight scala %} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD + +val training: RDD[LabeledPoint] = MLUtils.loadLibSVMData(sc, "mllib/data/sample_libsvm_data.txt") +{% endhighlight %} +
      + +
      +[`MLUtils.loadLibSVMData`](api/mllib/index.html#org.apache.spark.mllib.util.MLUtils$) reads training +examples stored in LIBSVM format. + +{% highlight java %} +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.rdd.RDDimport; + +RDD[LabeledPoint] training = MLUtils.loadLibSVMData(sc, "mllib/data/sample_libsvm_data.txt") +{% endhighlight %} +
      +
      + +## Local matrix + +A local matrix has integer-typed row and column indices and double-typed values, stored on a single +machine. MLlib supports dense matrix, whose entry values are stored in a single double array in +column major. For example, the following matrix `\[ \begin{pmatrix} +1.0 & 2.0 \\ +3.0 & 4.0 \\ +5.0 & 6.0 +\end{pmatrix} +\]` +is stored in a one-dimensional array `[1.0, 3.0, 5.0, 2.0, 4.0, 6.0]` with the matrix size `(3, 2)`. +We are going to add sparse matrix in the next release. + +
      +
      + +The base class of local matrices is +[`Matrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.Matrix), and we provide one +implementation: [`DenseMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.DenseMatrix). +Sparse matrix will be added in the next release. We recommend using the factory methods implemented +in [`Matrices`](api/mllib/index.html#org.apache.spark.mllib.linalg.Matrices) to create local +matrices. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.{Matrix, Matrices} + +// Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) +val dm: Matrix = Matrices.dense(3, 2, Array(1.0, 3.0, 5.0, 2.0, 4.0, 6.0)) +{% endhighlight %} +
      + +
      + +The base class of local matrices is +[`Matrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.Matrix), and we provide one +implementation: [`DenseMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.DenseMatrix). +Sparse matrix will be added in the next release. We recommend using the factory methods implemented +in [`Matrices`](api/mllib/index.html#org.apache.spark.mllib.linalg.Matrices) to create local +matrices. + +{% highlight java %} +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.Matrices; + +// Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) +Matrix dm = Matrices.dense(3, 2, new double[] {1.0, 3.0, 5.0, 2.0, 4.0, 6.0}); +{% endhighlight %} +
      + +
      + +## Distributed matrix + +A distributed matrix has long-typed row and column indices and double-typed values, stored +distributively in one or more RDDs. It is very important to choose the right format to store large +and distributed matrices. Converting a distributed matrix to a different format may require a +global shuffle, which is quite expensive. We implemented three types of distributed matrices in +this release and will add more types in the future. + +***Note*** + +The underlying RDDs of a distributed matrix must be deterministic, because we cache the matrix size. +It is always error-prone to have non-deterministic RDDs. + +### RowMatrix + +A `RowMatrix` is a row-oriented distributed matrix without meaningful row indices, backed by an RDD +of its rows, where each row is a local vector. This is similar to `data matrix` in the context of +multivariate statistics. Since each row is represented by a local vector, the number of columns is +limited by the integer range but it should be much smaller in practice. + +
      +
      + +A [`RowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) can be +created from an `RDD[Vector]` instance. Then we can compute its column summary statistics. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.distributed.RowMatrix + +val rows: RDD[Vector] = ... // an RDD of local vectors +// Create a RowMatrix from an RDD[Vector]. +val mat: RowMatrix = new RowMatrix(rows) + +// Get its size. +val m = mat.numRows() +val n = mat.numCols() +{% endhighlight %} +
      + +
      + +A [`RowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) can be +created from a `JavaRDD` instance. Then we can compute its column summary statistics. + +{% highlight java %} +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; + +JavaRDD rows = ... // a JavaRDD of local vectors +// Create a RowMatrix from an JavaRDD. +RowMatrix mat = new RowMatrix(rows.rdd()); + +// Get its size. +long m = mat.numRows(); +long n = mat.numCols(); +{% endhighlight %} +
      +
      + +#### Multivariate summary statistics + +We provide column summary statistics for `RowMatrix`. +If the number of columns is not large, say, smaller than 3000, you can also compute +the covariance matrix as a local matrix, which requires $\mathcal{O}(n^2)$ storage where $n$ is the +number of columns. The total CPU time is $\mathcal{O}(m n^2)$, where $m$ is the number of rows, +which could be faster if the rows are sparse. + +
      +
      + +`RowMatrix#computeColumnSummaryStatistics` returns an instance of +[`MultivariateStatisticalSummary`](api/mllib/index.html#org.apache.spark.mllib.stat.MultivariateStatisticalSummary), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.Matrix +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary + +val mat: RowMatrix = ... // a RowMatrix + +// Compute column summary statistics. +val summary: MultivariateStatisticalSummary = mat.computeColumnSummaryStatistics() +println(summary.mean) // a dense vector containing the mean value for each column +println(summary.variance) // column-wise variance +println(summary.numNonzers) // number of nonzeros in each column + +// Compute the covariance matrix. +val Cov: Matrix = mat.computeCovariance() +{% endhighlight %} +
      +
      + +### IndexedRowMatrix + +An `IndexedRowMatrix` is similar to a `RowMatrix` but with meaningful row indices. It is backed by +an RDD of indexed rows, which each row is represented by its index (long-typed) and a local vector. + +
      +
      + +An +[`IndexedRowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix) +can be created from an `RDD[IndexedRow]` instance, where +[`IndexedRow`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRow) is a +wrapper over `(Long, Vector)`. An `IndexedRowMatrix` can be converted to a `RowMatrix` by dropping +its row indices. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.distributed.{IndexedRow, IndexedRowMatrix, RowMatrix} + +val rows: RDD[IndexedRow] = ... // an RDD of indexed rows +// Create an IndexedRowMatrix from an RDD[IndexedRow]. +val mat: IndexedRowMatrix = new IndexedRowMatrix(rows) + +// Get its size. +val m = mat.numRows() +val n = mat.numCols() + +// Drop its row indices. +val rowMat: RowMatrix = mat.toRowMatrix() +{% endhighlight %} +
      + +
      + +An +[`IndexedRowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix) +can be created from an `JavaRDD` instance, where +[`IndexedRow`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRow) is a +wrapper over `(long, Vector)`. An `IndexedRowMatrix` can be converted to a `RowMatrix` by dropping +its row indices. + +{% highlight java %} +import org.apache.spark.mllib.linalg.distributed.IndexedRow; +import org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; + +JavaRDD[IndexedRow] rows = ... // a JavaRDD of indexed rows +// Create an IndexedRowMatrix from a JavaRDD. +IndexedRowMatrix mat = new IndexedRowMatrix(rows.rdd()); + +// Get its size. +long m = mat.numRows(); +long n = mat.numCols(); + +// Drop its row indices. +RowMatrix rowMat = mat.toRowMatrix(); +{% endhighlight %} +
      + +### CoordinateMatrix + +A `CoordinateMatrix` is a distributed matrix backed by an RDD of its entries. Each entry is a tuple +of `(i: Long, j: Long, value: Double)`, where `i` is the row index, `j` is the column index, and +`value` is the entry value. A `CoordinateMatrix` should be used only in the case when both +dimensions of the matrix are huge and the matrix is very sparse. + +
      +
      + +A +[`CoordinateMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.CoordinateMatrix) +can be created from an `RDD[MatrixEntry]` instance, where +[`MatrixEntry`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.MatrixEntry) is a +wrapper over `(Long, Long, Double)`. A `CoordinateMatrix` can be converted to a `IndexedRowMatrix` +with sparse rows by calling `toIndexedRowMatrix`. In this release, we do not provide other +computation for `CoordinateMatrix`. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.distributed.{CoordinateMatrix, MatrixEntry} + +val entries: RDD[MatrixEntry] = ... // an RDD of matrix entries +// Create a CoordinateMatrix from an RDD[MatrixEntry]. +val mat: CoordinateMatrix = new CoordinateMatrix(entries) + +// Get its size. +val m = mat.numRows() +val n = mat.numCols() + +// Convert it to an IndexRowMatrix whose rows are sparse vectors. +val indexedRowMatrix = mat.toIndexedRowMatrix() +{% endhighlight %} +
      + +
      + +A +[`CoordinateMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.CoordinateMatrix) +can be created from a `JavaRDD` instance, where +[`MatrixEntry`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.MatrixEntry) is a +wrapper over `(long, long, double)`. A `CoordinateMatrix` can be converted to a `IndexedRowMatrix` +with sparse rows by calling `toIndexedRowMatrix`. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.distributed.CoordinateMatrix; +import org.apache.spark.mllib.linalg.distributed.MatrixEntry; + +JavaRDD entries = ... // a JavaRDD of matrix entries +// Create a CoordinateMatrix from a JavaRDD. +CoordinateMatrix mat = new CoordinateMatrix(entries); + +// Get its size. +long m = mat.numRows(); +long n = mat.numCols(); + +// Convert it to an IndexRowMatrix whose rows are sparse vectors. +IndexedRowMatrix indexedRowMatrix = mat.toIndexedRowMatrix(); +{% endhighlight %} +
      +
      diff --git a/docs/mllib-classification-regression.md b/docs/mllib-classification-regression.md deleted file mode 100644 index 2e0fa093dccaa..0000000000000 --- a/docs/mllib-classification-regression.md +++ /dev/null @@ -1,568 +0,0 @@ ---- -layout: global -title: MLlib - Classification and Regression ---- - -* Table of contents -{:toc} - - -`\[ -\newcommand{\R}{\mathbb{R}} -\newcommand{\E}{\mathbb{E}} -\newcommand{\x}{\mathbf{x}} -\newcommand{\y}{\mathbf{y}} -\newcommand{\wv}{\mathbf{w}} -\newcommand{\av}{\mathbf{\alpha}} -\newcommand{\bv}{\mathbf{b}} -\newcommand{\N}{\mathbb{N}} -\newcommand{\id}{\mathbf{I}} -\newcommand{\ind}{\mathbf{1}} -\newcommand{\0}{\mathbf{0}} -\newcommand{\unit}{\mathbf{e}} -\newcommand{\one}{\mathbf{1}} -\newcommand{\zero}{\mathbf{0}} -\]` - - -# Supervised Machine Learning -Supervised machine learning is the setting where we are given a set of training data examples -`$\{\x_i\}$`, each example `$\x_i$` coming with a corresponding label `$y_i$`. -Given the training data `$\{(\x_i,y_i)\}$`, we want to learn a function to predict these labels. -The two most well known classes of methods are -[classification](http://en.wikipedia.org/wiki/Statistical_classification), and -[regression](http://en.wikipedia.org/wiki/Regression_analysis). -In classification, the label is a category (e.g. whether or not emails are spam), whereas in -regression, the label is real value, and we want our prediction to be as close to the true value -as possible. - -Supervised Learning involves executing a learning *Algorithm* on a set of *labeled* training -examples. The algorithm returns a trained *Model* (such as for example a linear function) that -can predict the label for new data examples for which the label is unknown. - -## Discriminative Training using Linear Methods - -### Mathematical Formulation -Many standard *machine learning* methods can be formulated as a convex optimization problem, i.e. -the task of finding a minimizer of a convex function `$f$` that depends on a variable vector -`$\wv$` (called `weights` in the code), which has `$d$` entries. -Formally, we can write this as the optimization problem `$\min_{\wv \in\R^d} \; f(\wv)$`, where -the objective function is of the form -`\begin{equation} - f(\wv) := - \lambda\, R(\wv) + - \frac1n \sum_{i=1}^n L(\wv;\x_i,y_i) - \label{eq:regPrimal} - \ . -\end{equation}` -Here the vectors `$\x_i\in\R^d$` are the training data examples, for `$1\le i\le n$`, and -`$y_i\in\R$` are their corresponding labels, which we want to predict. - -The objective function `$f$` has two parts: -The *loss-function* measures the error of the model on the training data. The loss-function -`$L(\wv;.)$` must be a convex function in `$\wv$`. -The purpose of the [regularizer](http://en.wikipedia.org/wiki/Regularization_(mathematics)) is to -encourage simple models, by punishing the complexity of the model `$\wv$`, in order to e.g. avoid -over-fitting. -Usually, the regularizer `$R(.)$` is chosen as either the standard (Euclidean) L2-norm, `$R(\wv) -:= \frac{1}{2}\|\wv\|^2$`, or the L1-norm, `$R(\wv) := \|\wv\|_1$`, see -[below](#using-different-regularizers) for more details. - -The fixed regularization parameter `$\lambda\ge0$` (`regParam` in the code) defines the trade-off -between the two goals of small loss and small model complexity. - - -### Binary Classification - -**Input:** Datapoints `$\x_i\in\R^{d}$`, labels `$y_i\in\{+1,-1\}$`, for `$1\le i\le n$`. - -**Distributed Datasets.** -For all currently implemented optimization methods for classification, the data must be -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$`. - -#### Support Vector Machine -The linear [Support Vector Machine (SVM)](http://en.wikipedia.org/wiki/Support_vector_machine) -has become a standard choice for classification tasks. -Here the loss function in formulation `$\eqref{eq:regPrimal}$` is given by the hinge-loss -`\[ -L(\wv;\x_i,y_i) := \max \{0, 1-y_i \wv^T \x_i \} \ . -\]` - -By default, SVMs are trained with an L2 regularization, which gives rise to the large-margin -interpretation if these classifiers. We also support alternative L1 regularization. In this case, -the primal optimization problem becomes an [LP](http://en.wikipedia.org/wiki/Linear_programming). - -#### Logistic Regression -Despite its name, [Logistic Regression](http://en.wikipedia.org/wiki/Logistic_regression) is a -binary classification method, again when the labels are given by binary values -`$y_i\in\{+1,-1\}$`. The logistic loss function in formulation `$\eqref{eq:regPrimal}$` is -defined as -`\[ -L(\wv;\x_i,y_i) := \log(1+\exp( -y_i \wv^T \x_i)) \ . -\]` - - -### Linear Regression (Least Squares, Lasso and Ridge Regression) - -**Input:** Data matrix `$A\in\R^{n\times d}$`, right hand side vector `$\y\in\R^n$`. - -**Distributed Datasets.** -For all currently implemented optimization methods for regression, the data matrix -`$A\in\R^{n\times d}$` must be distributed between the worker machines *by rows* of `$A$`. In -other words, the input distributed dataset -([RDD](scala-programming-guide.html#resilient-distributed-datasets-rdds)) must be the set of the -`$n$` rows `$A_{i:}$` of `$A$`. - -Least Squares Regression refers to the setting where we try to fit a vector `$\y\in\R^n$` by -linear combination of our observed data `$A\in\R^{n\times d}$`, which is given as a matrix. - -It comes in 3 flavors: - -#### Least Squares -Plain old [least squares](http://en.wikipedia.org/wiki/Least_squares) linear regression is the -problem of minimizing - `\[ f_{\text{LS}}(\wv) := \frac1n \|A\wv-\y\|_2^2 \ . \]` - -#### Lasso -The popular [Lasso](http://en.wikipedia.org/wiki/Lasso_(statistics)#Lasso_method) (alternatively -also known as `$L_1$`-regularized least squares regression) is given by - `\[ f_{\text{Lasso}}(\wv) := \frac1n \|A\wv-\y\|_2^2 + \lambda \|\wv\|_1 \ . \]` - -#### Ridge Regression -[Ridge regression](http://en.wikipedia.org/wiki/Ridge_regression) uses the same loss function but -with a L2 regularizer term: - `\[ f_{\text{Ridge}}(\wv) := \frac1n \|A\wv-\y\|_2^2 + \frac{\lambda}{2}\|\wv\|^2 \ . \]` - -**Loss Function.** -For all 3, the loss function (i.e. the measure of model fit) is given by the squared deviations -from the right hand side `$\y$`. -`\[ -\frac1n \|A\wv-\y\|_2^2 -= \frac1n \sum_{i=1}^n (A_{i:} \wv - y_i )^2 -= \frac1n \sum_{i=1}^n L(\wv;\x_i,y_i) -\]` -This is also known as the [mean squared error](http://en.wikipedia.org/wiki/Mean_squared_error). -In our generic problem formulation `$\eqref{eq:regPrimal}$`, this means the loss function is -`$L(\wv;\x_i,y_i) := (A_{i:} \wv - y_i )^2$`, each depending only on a single row `$A_{i:}$` of -the data matrix `$A$`. - - -### Using Different Regularizers - -As we have mentioned above, the purpose of *regularizer* in `$\eqref{eq:regPrimal}$` is to -encourage simple models, by punishing the complexity of the model `$\wv$`, in order to e.g. avoid -over-fitting. -All machine learning methods for classification and regression that we have mentioned above are -of interest for different types of regularization, the 3 most common ones being - -* **L2-Regularization.** -`$R(\wv) := \frac{1}{2}\|\wv\|^2$`. -This regularizer is most commonly used for SVMs, logistic regression and ridge regression. - -* **L1-Regularization.** -`$R(\wv) := \|\wv\|_1$`. The L1 norm `$\|\wv\|_1$` is the sum of the absolut values of the -entries of a vector `$\wv$`. -This regularizer is most commonly used for sparse methods, and feature selection, such as the -Lasso. - -* **Non-Regularized.** -`$R(\wv):=0$`. -Of course we can also train the models without any regularization, or equivalently by setting the -regularization parameter `$\lambda:=0$`. - -The optimization problems of the form `$\eqref{eq:regPrimal}$` with convex regularizers such as -the 3 mentioned here can be conveniently optimized with gradient descent type methods (such as -SGD) which is implemented in `MLlib` currently, and explained in the next section. - - -### Optimization Methods Working on the Primal Formulation - -**Stochastic subGradient Descent (SGD).** -For optimization objectives `$f$` written as a sum, *stochastic subgradient descent (SGD)* can be -an efficient choice of optimization method, as we describe in the optimization section in more detail. -Because all methods considered here fit into the optimization formulation -`$\eqref{eq:regPrimal}$`, this is especially natural, because the loss is written as an average -of the individual losses coming from each datapoint. - -Picking one datapoint `$i\in[1..n]$` uniformly at random, we obtain a stochastic subgradient of -`$\eqref{eq:regPrimal}$`, with respect to `$\wv$` as follows: -`\[ -f'_{\wv,i} := L'_{\wv,i} + \lambda\, R'_\wv \ , -\]` -where `$L'_{\wv,i} \in \R^d$` is a subgradient of the part of the loss function determined by the -`$i$`-th datapoint, that is `$L'_{\wv,i} \in \frac{\partial}{\partial \wv} L(\wv;\x_i,y_i)$`. -Furthermore, `$R'_\wv$` is a subgradient of the regularizer `$R(\wv)$`, i.e. `$R'_\wv \in -\frac{\partial}{\partial \wv} R(\wv)$`. The term `$R'_\wv$` does not depend on which random -datapoint is picked. - - - -**Gradients.** -The following table summarizes the gradients (or subgradients) of all loss functions and -regularizers that we currently support: - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
      FunctionStochastic (Sub)Gradient
      SVM Hinge Loss$L(\wv;\x_i,y_i) := \max \{0, 1-y_i \wv^T \x_i \}$$L'_{\wv,i} = \begin{cases}-y_i \x_i & \text{if $y_i \wv^T \x_i <1$}, \\ 0 & -\text{otherwise}.\end{cases}$
      Logistic Loss$L(\wv;\x_i,y_i) := \log(1+\exp( -y_i \wv^T \x_i))$$L'_{\wv,i} = -y_i \x_i \left(1-\frac1{1+\exp(-y_i \wv^T \x_i)} \right)$
      Least Squares Loss$L(\wv;\x_i,y_i) := (A_{i:} \wv - y_i)^2$$L'_{\wv,i} = 2 A_{i:}^T (A_{i:} \wv - y_i)$
      Non-Regularized$R(\wv) := 0$$R'_\wv = \0$
      L2 Regularizer$R(\wv) := \frac{1}{2}\|\wv\|^2$$R'_\wv = \wv$
      L1 Regularizer$R(\wv) := \|\wv\|_1$$R'_\wv = \mathop{sign}(\wv)$
      - -Here `$\mathop{sign}(\wv)$` is the vector consisting of the signs (`$\pm1$`) of all the entries -of `$\wv$`. -Also, note that `$A_{i:} \in \R^d$` is a row-vector, but the gradient is a column vector. - -## Decision Tree Classification and Regression - -Decision trees and their ensembles are popular methods for the machine learning tasks of classification and regression. Decision trees are widely used since they are easy to interpret, handle categorical variables, extend to the multi-class classification setting, do not require feature scaling and are able to capture non-linearities and feature interactions. Tree ensemble algorithms such as decision forest and boosting are among the top performers for classification and regression tasks. - -### Basic Algorithm - -The decision tree is a greedy algorithm that performs a recursive binary partitioning of the feature space by choosing a single element from the *best split set* where each element of the set maximimizes the information gain at a tree node. In other words, the split chosen at each tree node is chosen from the set `$\underset{s}{\operatorname{argmax}} IG(D,s)$` where `$IG(D,s)$` is the information gain when a split `$s$` is applied to a dataset `$D$`. - -#### Node Impurity and Information Gain - -The *node impurity* is a measure of the homogeneity of the labels at the node. The current implementation provides two impurity measures for classification (Gini index and entropy) and one impurity measure for regression (variance). - - - - - - - - - - - - - - - - -
      ImpurityTaskFormulaDescription
      Gini indexClassification$\sum_{i=1}^{M} f_i(1-f_i)$$f_i$ is the frequency of label $i$ at a node and $M$ is the number of unique labels.
      EntropyClassification$\sum_{i=1}^{M} -f_ilog(f_i)$$f_i$ is the frequency of label $i$ at a node and $M$ is the number of unique labels.
      VarianceClassification$\frac{1}{n} \sum_{i=1}^{N} (x_i - \mu)^2$$y_i$ is label for an instance, $N$ is the number of instances and $\mu$ is the mean given by $\frac{1}{N} \sum_{i=1}^n x_i$.
      - -The *information gain* is the difference in the parent node impurity and the weighted sum of the two child node impurities. Assuming that a split $s$ partitions the dataset `$D$` of size `$N$` into two datasets `$D_{left}$` and `$D_{right}$` of sizes `$N_{left}$` and `$N_{right}$`, respectively: - -`$IG(D,s) = Impurity(D) - \frac{N_{left}}{N} Impurity(D_{left}) - \frac{N_{right}}{N} Impurity(D_{right})$` - -#### Split Candidates - -**Continuous Features** - -For small datasets in single machine implementations, the split candidates for each continuous feature are typically the unique values for the feature. Some implementations sort the feature values and then use the ordered unique values as split candidates for faster tree calculations. - -Finding ordered unique feature values is computationally intensive for large distributed datasets. One can get an approximate set of split candidates by performing a quantile calculation over a sampled fraction of the data. The ordered splits create "bins" and the maximum number of such bins can be specified using the `maxBins` parameters. - -Note that the number of bins cannot be greater than the number of instances `$N$` (a rare scenario since the default `maxBins` value is 100). The tree algorithm automatically reduces the number of bins if the condition is not satisfied. - -**Categorical Features** - -For `$M$` categorical features, one could come up with `$2^M-1$` split candidates. However, for binary classification, the number of split candidates can be reduced to `$M-1$` by ordering the categorical feature values by the proportion of labels falling in one of the two classes (see Section 9.2.4 in [Elements of Statistical Machine Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for details). For example, for a binary classification problem with one categorical feature with three categories A, B and C with corresponding proportion of label 1 as 0.2, 0.6 and 0.4, the categorical features are orded as A followed by C followed B or A, B, C. The two split candidates are A \| C, B and A , B \| C where \| denotes the split. - -#### Stopping Rule - -The recursive tree construction is stopped at a node when one of the two conditions is met: - -1. The node depth is equal to the `maxDepth` training paramemter -2. No split candidate leads to an information gain at the node. - -### Practical Limitations - -The tree implementation stores an Array[Double] of size *O(#features \* #splits \* 2^maxDepth)* in memory for aggregating histograms over partitions. The current implementation might not scale to very deep trees since the memory requirement grows exponentially with tree depth. - -Please drop us a line if you encounter any issues. We are planning to solve this problem in the near future and real-world examples will be great. - - -## Implementation in MLlib - -#### Linear Methods - -For both classification and regression algorithms with convex loss functions, `MLlib` implements a simple distributed version of -stochastic subgradient descent (SGD), building on the underlying gradient descent primitive (as -described in the -optimization section). -All provided algorithms take as input a regularization parameter (`regParam`) along with various -parameters associated with stochastic gradient -descent (`stepSize`, `numIterations`, `miniBatchFraction`). -For each of them, we support all 3 possible regularizations (none, L1 or L2). - -Available algorithms for binary classification: - -* [SVMWithSGD](api/scala/index.html#org.apache.spark.mllib.classification.SVMWithSGD) -* [LogisticRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) - -Available algorithms for linear regression: - -* [LinearRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD) -* [RidgeRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) -* [LassoWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.LassoWithSGD) - -Behind the scenes, all above methods use the SGD implementation from the -gradient descent primitive in MLlib, see the -optimization part: - -* [GradientDescent](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) - -#### Tree-based Methods - -The decision tree algorithm supports binary classification and regression: - -* [DecisionTee](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) - - -# Usage in Scala - -Following code snippets can be executed in `spark-shell`. - -## Linear Methods - - -#### Binary Classification - -The following code snippet illustrates how to load a sample dataset, execute a -training algorithm on this training data using a static method in the algorithm -object, and make predictions with the resulting model to compute the training -error. - -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.classification.SVMWithSGD -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors - -// Load and parse the data file -val data = sc.textFile("mllib/data/sample_svm_data.txt") -val parsedData = data.map { line => - val parts = line.split(' ').map(_.toDouble) - LabeledPoint(parts(0), Vectors.dense(parts.tail)) -} - -// Run training algorithm to build the model -val numIterations = 100 -val model = SVMWithSGD.train(parsedData, numIterations) - -// Evaluate model on training examples and compute training error -val labelAndPreds = parsedData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count -println("Training Error = " + trainErr) -{% endhighlight %} - - -The `SVMWithSGD.train()` method by default performs L2 regularization with the -regularization parameter set to 1.0. If we want to configure this algorithm, we -can customize `SVMWithSGD` further by creating a new object directly and -calling setter methods. All other MLlib algorithms support customization in -this way as well. For example, the following code produces an L1 regularized -variant of SVMs with regularization parameter set to 0.1, and runs the training -algorithm for 200 iterations. - -{% highlight scala %} -import org.apache.spark.mllib.optimization.L1Updater - -val svmAlg = new SVMWithSGD() -svmAlg.optimizer.setNumIterations(200) - .setRegParam(0.1) - .setUpdater(new L1Updater) -val modelL1 = svmAlg.run(parsedData) -{% endhighlight %} - -#### Linear Regression - -The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. -The example then uses LinearRegressionWithSGD to build a simple linear model to predict label -values. We compute the Mean Squared Error at the end to evaluate -[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). - -{% highlight scala %} -import org.apache.spark.mllib.regression.LinearRegressionWithSGD -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors - -// Load and parse the data -val data = sc.textFile("mllib/data/ridge-data/lpsa.data") -val parsedData = data.map { line => - val parts = line.split(',') - LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) -} - -// Building the model -val numIterations = 100 -val model = LinearRegressionWithSGD.train(parsedData, numIterations) - -// Evaluate model on training examples and compute training error -val valuesAndPreds = parsedData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.reduce(_ + _) / valuesAndPreds.count -println("training Mean Squared Error = " + MSE) -{% endhighlight %} - - -Similarly you can use RidgeRegressionWithSGD and LassoWithSGD and compare training -[Mean Squared Errors](http://en.wikipedia.org/wiki/Mean_squared_error). - -## Decision Tree - -#### Classification - -The example below demonstrates how to load a CSV file, parse it as an RDD of LabeledPoint and then perform classification using a decision tree using Gini index as an impurity measure and a maximum tree depth of 5. The training error is calculated to measure the algorithm accuracy. - -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.tree.DecisionTree -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.impurity.Gini - -// Load and parse the data file -val data = sc.textFile("mllib/data/sample_tree_data.csv") -val parsedData = data.map { line => - val parts = line.split(',').map(_.toDouble) - LabeledPoint(parts(0), Vectors.dense(parts.tail)) -} - -// Run training algorithm to build the model -val maxDepth = 5 -val model = DecisionTree.train(parsedData, Classification, Gini, maxDepth) - -// Evaluate model on training examples and compute training error -val labelAndPreds = parsedData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count -println("Training Error = " + trainErr) -{% endhighlight %} - -#### Regression - -The example below demonstrates how to load a CSV file, parse it as an RDD of LabeledPoint and then perform regression using a decision tree using variance as an impurity measure and a maximum tree depth of 5. The Mean Squared Error is computed at the end to evaluate -[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). - -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.tree.DecisionTree -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.impurity.Variance - -// Load and parse the data file -val data = sc.textFile("mllib/data/sample_tree_data.csv") -val parsedData = data.map { line => - val parts = line.split(',').map(_.toDouble) - LabeledPoint(parts(0), Vectors.dense(parts.tail)) -} - -// Run training algorithm to build the model -val maxDepth = 5 -val model = DecisionTree.train(parsedData, Regression, Variance, maxDepth) - -// Evaluate model on training examples and compute training error -val valuesAndPreds = parsedData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.reduce(_ + _)/valuesAndPreds.count -println("training Mean Squared Error = " + MSE) -{% endhighlight %} - - -# Usage in Java - -All of MLlib's methods use Java-friendly types, so you can import and call them there the same -way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the -Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by -calling `.rdd()` on your `JavaRDD` object. - -# Usage in Python - -Following examples can be tested in the PySpark shell. - -## Linear Methods - -### Binary Classification -The following example shows how to load a sample dataset, build Logistic Regression model, -and make predictions with the resulting model to compute the training error. - -{% highlight python %} -from pyspark.mllib.classification import LogisticRegressionWithSGD -from pyspark.mllib.regression import LabeledPoint -from numpy import array - -# Load and parse the data -def parsePoint(line): - values = [float(x) for x in line.split(' ')] - return LabeledPoint(values[0], values[1:]) - -data = sc.textFile("mllib/data/sample_svm_data.txt") -parsedData = data.map(parsePoint) - -# Build the model -model = LogisticRegressionWithSGD.train(parsedData) - -# Evaluating the model on training data -labelsAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) -trainErr = labelsAndPreds.filter(lambda (v, p): v != p).count() / float(parsedData.count()) -print("Training Error = " + str(trainErr)) -{% endhighlight %} - -### Linear Regression -The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. -The example then uses LinearRegressionWithSGD to build a simple linear model to predict label -values. We compute the Mean Squared Error at the end to evaluate -[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). - -{% highlight python %} -from pyspark.mllib.regression import LabeledPoint, LinearRegressionWithSGD -from numpy import array - -# Load and parse the data -def parsePoint(line): - values = [float(x) for x in line.replace(',', ' ').split(' ')] - return LabeledPoint(values[0], values[1:]) - -data = sc.textFile("mllib/data/ridge-data/lpsa.data") -parsedData = data.map(parsePoint) - -# Build the model -model = LinearRegressionWithSGD.train(parsedData) - -# Evaluate the model on training data -valuesAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) -MSE = valuesAndPreds.map(lambda (v, p): (v - p)**2).reduce(lambda x, y: x + y) / valuesAndPreds.count() -print("Mean Squared Error = " + str(MSE)) -{% endhighlight %} diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 0359c67157168..b3293afe408d0 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -1,19 +1,21 @@ --- layout: global -title: MLlib - Clustering +title: MLlib - Clustering --- * Table of contents {:toc} -# Clustering +## Clustering Clustering is an unsupervised learning problem whereby we aim to group subsets of entities with one another based on some notion of similarity. Clustering is often used for exploratory analysis and/or as a component of a hierarchical supervised learning pipeline (in which distinct classifiers or regression -models are trained for each cluster). MLlib supports +models are trained for each cluster). + +MLlib supports [k-means](http://en.wikipedia.org/wiki/K-means_clustering) clustering, one of the most commonly used clustering algorithms that clusters the data points into predfined number of clusters. The MLlib implementation includes a parallelized @@ -31,17 +33,14 @@ a given dataset, the algorithm returns the best clustering result). * *initializiationSteps* determines the number of steps in the k-means\|\| algorithm. * *epsilon* determines the distance threshold within which we consider k-means to have converged. -Available algorithms for clustering: - -* [KMeans](api/scala/index.html#org.apache.spark.mllib.clustering.KMeans) - - - -# Usage in Scala +## Examples +
      +
      Following code snippets can be executed in `spark-shell`. -In the following example after loading and parsing data, we use the KMeans object to cluster the data +In the following example after loading and parsing data, we use the +[`KMeans`](api/mllib/index.html#org.apache.spark.mllib.clustering.KMeans) object to cluster the data into two clusters. The number of desired clusters is passed to the algorithm. We then compute Within Set Sum of Squared Error (WSSSE). You can reduce this error measure by increasing *k*. In fact the optimal *k* is usually one where there is an "elbow" in the WSSSE graph. @@ -63,22 +62,22 @@ val clusters = KMeans.train(parsedData, numClusters, numIterations) val WSSSE = clusters.computeCost(parsedData) println("Within Set Sum of Squared Errors = " + WSSSE) {% endhighlight %} +
      - -# Usage in Java - +
      All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by calling `.rdd()` on your `JavaRDD` object. +
      -# Usage in Python +
      Following examples can be tested in the PySpark shell. -In the following example after loading and parsing data, we use the KMeans object to cluster the data -into two clusters. The number of desired clusters is passed to the algorithm. We then compute Within -Set Sum of Squared Error (WSSSE). You can reduce this error measure by increasing *k*. In fact the -optimal *k* is usually one where there is an "elbow" in the WSSSE graph. +In the following example after loading and parsing data, we use the KMeans object to cluster the +data into two clusters. The number of desired clusters is passed to the algorithm. We then compute +Within Set Sum of Squared Error (WSSSE). You can reduce this error measure by increasing *k*. In +fact the optimal *k* is usually one where there is an "elbow" in the WSSSE graph. {% highlight python %} from pyspark.mllib.clustering import KMeans @@ -91,7 +90,7 @@ parsedData = data.map(lambda line: array([float(x) for x in line.split(' ')])) # Build the model (cluster the data) clusters = KMeans.train(parsedData, 2, maxIterations=10, - runs=10, initialization_mode="random") + runs=10, initializationMode="random") # Evaluate clustering by computing Within Set Sum of Squared Errors def error(point): @@ -101,7 +100,6 @@ def error(point): WSSSE = parsedData.map(lambda point: error(point)).reduce(lambda x, y: x + y) print("Within Set Sum of Squared Error = " + str(WSSSE)) {% endhighlight %} +
      -Similarly you can use RidgeRegressionWithSGD and LassoWithSGD and compare training Mean Squared -Errors. - +
      diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index 2f1f5f3856efe..79f5e3a7ca4fb 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -1,12 +1,12 @@ --- layout: global -title: MLlib - Collaborative Filtering +title: MLlib - Collaborative Filtering --- * Table of contents {:toc} -# Collaborative Filtering +## Collaborative filtering [Collaborative filtering](http://en.wikipedia.org/wiki/Recommender_system#Collaborative_filtering) is commonly used for recommender systems. These techniques aim to fill in the @@ -14,44 +14,43 @@ missing entries of a user-item association matrix. MLlib currently supports model-based collaborative filtering, in which users and products are described by a small set of latent factors that can be used to predict missing entries. In particular, we implement the [alternating least squares -(ALS)](http://www2.research.att.com/~volinsky/papers/ieeecomputer.pdf) +(ALS)](http://dl.acm.org/citation.cfm?id=1608614) algorithm to learn these latent factors. The implementation in MLlib has the following parameters: -* *numBlocks* is the number of blacks used to parallelize computation (set to -1 to auto-configure). +* *numBlocks* is the number of blocks used to parallelize computation (set to -1 to auto-configure). * *rank* is the number of latent factors in our model. * *iterations* is the number of iterations to run. * *lambda* specifies the regularization parameter in ALS. -* *implicitPrefs* specifies whether to use the *explicit feedback* ALS variant or one adapted for *implicit feedback* data -* *alpha* is a parameter applicable to the implicit feedback variant of ALS that governs the *baseline* confidence in preference observations +* *implicitPrefs* specifies whether to use the *explicit feedback* ALS variant or one adapted for + *implicit feedback* data. +* *alpha* is a parameter applicable to the implicit feedback variant of ALS that governs the + *baseline* confidence in preference observations. -## Explicit vs Implicit Feedback +### Explicit vs. implicit feedback The standard approach to matrix factorization based collaborative filtering treats the entries in the user-item matrix as *explicit* preferences given by the user to the item. -It is common in many real-world use cases to only have access to *implicit feedback* -(e.g. views, clicks, purchases, likes, shares etc.). The approach used in MLlib to deal with -such data is taken from -[Collaborative Filtering for Implicit Feedback Datasets](http://www2.research.att.com/~yifanhu/PUB/cf.pdf). -Essentially instead of trying to model the matrix of ratings directly, this approach treats the data as -a combination of binary preferences and *confidence values*. The ratings are then related -to the level of confidence in observed user preferences, rather than explicit ratings given to items. -The model then tries to find latent factors that can be used to predict the expected preference of a user -for an item. +It is common in many real-world use cases to only have access to *implicit feedback* (e.g. views, +clicks, purchases, likes, shares etc.). The approach used in MLlib to deal with such data is taken +from +[Collaborative Filtering for Implicit Feedback Datasets](http://dx.doi.org/10.1109/ICDM.2008.22). +Essentially instead of trying to model the matrix of ratings directly, this approach treats the data +as a combination of binary preferences and *confidence values*. The ratings are then related to the +level of confidence in observed user preferences, rather than explicit ratings given to items. The +model then tries to find latent factors that can be used to predict the expected preference of a +user for an item. -Available algorithms for collaborative filtering: +## Examples -* [ALS](api/scala/index.html#org.apache.spark.mllib.recommendation.ALS) - - -# Usage in Scala - -Following code snippets can be executed in `spark-shell`. +
      +
      In the following example we load rating data. Each row consists of a user, a product and a rating. -We use the default ALS.train() method which assumes ratings are explicit. We evaluate the recommendation -model by measuring the Mean Squared Error of rating prediction. +We use the default [ALS.train()](api/mllib/index.html#org.apache.spark.mllib.recommendation.ALS$) +method which assumes ratings are explicit. We evaluate the +recommendation model by measuring the Mean Squared Error of rating prediction. {% highlight scala %} import org.apache.spark.mllib.recommendation.ALS @@ -64,8 +63,9 @@ val ratings = data.map(_.split(',') match { }) // Build the recommendation model using ALS +val rank = 10 val numIterations = 20 -val model = ALS.train(ratings, 1, 20, 0.01) +val model = ALS.train(ratings, rank, numIterations, 0.01) // Evaluate the model on rating data val usersProducts = ratings.map{ case Rating(user, product, rate) => (user, product)} @@ -85,19 +85,19 @@ If the rating matrix is derived from other source of information (i.e., it is in other signals), you can use the trainImplicit method to get better results. {% highlight scala %} -val model = ALS.trainImplicit(ratings, 1, 20, 0.01) +val alpha = 0.01 +val model = ALS.trainImplicit(ratings, rank, numIterations, alpha) {% endhighlight %} +
      -# Usage in Java - +
      All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by calling `.rdd()` on your `JavaRDD` object. +
      -# Usage in Python -Following examples can be tested in the PySpark shell. - +
      In the following example we load rating data. Each row consists of a user, a product and a rating. We use the default ALS.train() method which assumes ratings are explicit. We evaluate the recommendation by measuring the Mean Squared Error of rating prediction. @@ -111,7 +111,9 @@ data = sc.textFile("mllib/data/als/test.data") ratings = data.map(lambda line: array([float(x) for x in line.split(',')])) # Build the recommendation model using Alternating Least Squares -model = ALS.train(ratings, 1, 20) +rank = 10 +numIterations = 20 +model = ALS.train(ratings, rank, numIterations) # Evaluate the model on training data testdata = ratings.map(lambda p: (int(p[0]), int(p[1]))) @@ -126,5 +128,13 @@ signals), you can use the trainImplicit method to get better results. {% highlight python %} # Build the recommendation model using Alternating Least Squares based on implicit ratings -model = ALS.trainImplicit(ratings, 1, 20) +model = ALS.trainImplicit(ratings, rank, numIterations, alpha = 0.01) {% endhighlight %} +
      + +
      + +## Tutorial + +[AMP Camp](http://ampcamp.berkeley.edu/) provides a hands-on tutorial for +[personalized movie recommendation with MLlib](http://ampcamp.berkeley.edu/big-data-mini-course/movie-recommendation-with-mllib.html). diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md new file mode 100644 index 0000000000000..0693766990732 --- /dev/null +++ b/docs/mllib-decision-tree.md @@ -0,0 +1,185 @@ +--- +layout: global +title: MLlib - Decision Tree +--- + +* Table of contents +{:toc} + +Decision trees and their ensembles are popular methods for the machine learning tasks of +classification and regression. Decision trees are widely used since they are easy to interpret, +handle categorical variables, extend to the multiclass classification setting, do not require +feature scaling and are able to capture nonlinearities and feature interactions. Tree ensemble +algorithms such as decision forest and boosting are among the top performers for classification and +regression tasks. + +## Basic algorithm + +The decision tree is a greedy algorithm that performs a recursive binary partitioning of the feature +space by choosing a single element from the *best split set* where each element of the set maximizes +the information gain at a tree node. In other words, the split chosen at each tree node is chosen +from the set `$\underset{s}{\operatorname{argmax}} IG(D,s)$` where `$IG(D,s)$` is the information +gain when a split `$s$` is applied to a dataset `$D$`. + +### Node impurity and information gain + +The *node impurity* is a measure of the homogeneity of the labels at the node. The current +implementation provides two impurity measures for classification (Gini impurity and entropy) and one +impurity measure for regression (variance). + + + + + + + + + + + + + + + + + + + + + + +
      ImpurityTaskFormulaDescription
      Gini impurityClassification$\sum_{i=1}^{M} f_i(1-f_i)$$f_i$ is the frequency of label $i$ at a node and $M$ is the number of unique labels.
      EntropyClassification$\sum_{i=1}^{M} -f_ilog(f_i)$$f_i$ is the frequency of label $i$ at a node and $M$ is the number of unique labels.
      VarianceRegression$\frac{1}{n} \sum_{i=1}^{N} (x_i - \mu)^2$$y_i$ is label for an instance, + $N$ is the number of instances and $\mu$ is the mean given by $\frac{1}{N} \sum_{i=1}^n x_i$.
      + +The *information gain* is the difference in the parent node impurity and the weighted sum of the two +child node impurities. Assuming that a split $s$ partitions the dataset `$D$` of size `$N$` into two +datasets `$D_{left}$` and `$D_{right}$` of sizes `$N_{left}$` and `$N_{right}$`, respectively: + +`$IG(D,s) = Impurity(D) - \frac{N_{left}}{N} Impurity(D_{left}) - \frac{N_{right}}{N} Impurity(D_{right})$` + +### Split candidates + +**Continuous features** + +For small datasets in single machine implementations, the split candidates for each continuous +feature are typically the unique values for the feature. Some implementations sort the feature +values and then use the ordered unique values as split candidates for faster tree calculations. + +Finding ordered unique feature values is computationally intensive for large distributed +datasets. One can get an approximate set of split candidates by performing a quantile calculation +over a sampled fraction of the data. The ordered splits create "bins" and the maximum number of such +bins can be specified using the `maxBins` parameters. + +Note that the number of bins cannot be greater than the number of instances `$N$` (a rare scenario +since the default `maxBins` value is 100). The tree algorithm automatically reduces the number of +bins if the condition is not satisfied. + +**Categorical features** + +For `$M$` categorical features, one could come up with `$2^M-1$` split candidates. However, for +binary classification, the number of split candidates can be reduced to `$M-1$` by ordering the +categorical feature values by the proportion of labels falling in one of the two classes (see +Section 9.2.4 in +[Elements of Statistical Machine Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for +details). For example, for a binary classification problem with one categorical feature with three +categories A, B and C with corresponding proportion of label 1 as 0.2, 0.6 and 0.4, the categorical +features are orded as A followed by C followed B or A, B, C. The two split candidates are A \| C, B +and A , B \| C where \| denotes the split. + +### Stopping rule + +The recursive tree construction is stopped at a node when one of the two conditions is met: + +1. The node depth is equal to the `maxDepth` training parammeter +2. No split candidate leads to an information gain at the node. + +### Practical limitations + +1. The tree implementation stores an Array[Double] of size *O(#features \* #splits \* 2^maxDepth)* + in memory for aggregating histograms over partitions. The current implementation might not scale + to very deep trees since the memory requirement grows exponentially with tree depth. +2. The implemented algorithm reads both sparse and dense data. However, it is not optimized for + sparse input. +3. Python is not supported in this release. + +We are planning to solve these problems in the near future. Please drop us a line if you encounter +any issues. + +## Examples + +### Classification + +The example below demonstrates how to load a CSV file, parse it as an RDD of `LabeledPoint` and then +perform classification using a decision tree using Gini impurity as an impurity measure and a +maximum tree depth of 5. The training error is calculated to measure the algorithm accuracy. + +
      +
      +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.tree.DecisionTree +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.impurity.Gini + +// Load and parse the data file +val data = sc.textFile("mllib/data/sample_tree_data.csv") +val parsedData = data.map { line => + val parts = line.split(',').map(_.toDouble) + LabeledPoint(parts(0), Vectors.dense(parts.tail)) +} + +// Run training algorithm to build the model +val maxDepth = 5 +val model = DecisionTree.train(parsedData, Classification, Gini, maxDepth) + +// Evaluate model on training examples and compute training error +val labelAndPreds = parsedData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count +println("Training Error = " + trainErr) +{% endhighlight %} +
      +
      + +### Regression + +The example below demonstrates how to load a CSV file, parse it as an RDD of `LabeledPoint` and then +perform regression using a decision tree using variance as an impurity measure and a maximum tree +depth of 5. The Mean Squared Error (MSE) is computed at the end to evaluate +[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). + +
      +
      +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.tree.DecisionTree +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.impurity.Variance + +// Load and parse the data file +val data = sc.textFile("mllib/data/sample_tree_data.csv") +val parsedData = data.map { line => + val parts = line.split(',').map(_.toDouble) + LabeledPoint(parts(0), Vectors.dense(parts.tail)) +} + +// Run training algorithm to build the model +val maxDepth = 5 +val model = DecisionTree.train(parsedData, Regression, Variance, maxDepth) + +// Evaluate model on training examples and compute training error +val valuesAndPreds = parsedData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.reduce(_ + _)/valuesAndPreds.count +println("training Mean Squared Error = " + MSE) +{% endhighlight %} +
      +
      diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md new file mode 100644 index 0000000000000..4e9ecf7c006fa --- /dev/null +++ b/docs/mllib-dimensionality-reduction.md @@ -0,0 +1,86 @@ +--- +layout: global +title: MLlib - Dimensionality Reduction +--- + +* Table of contents +{:toc} + +[Dimensionality reduction](http://en.wikipedia.org/wiki/Dimensionality_reduction) is the process +of reducing the number of variables under consideration. +It is used to extract latent features from raw and noisy features, +or compress data while maintaining the structure. +In this release, we provide preliminary support for dimensionality reduction on tall-and-skinny matrices. + +## Singular value decomposition (SVD) + +[Singular value decomposition (SVD)](http://en.wikipedia.org/wiki/Singular_value_decomposition) +factorizes a matrix into three matrices: $U$, $\Sigma$, and $V$ such that + +`\[ +A = U \Sigma V^T, +\]` + +where + +* $U$ is an orthonormal matrix, whose columns are called left singular vectors, +* $\Sigma$ is a diagonal matrix with non-negative diagonals in descending order, + whose diagonals are called singular values, +* $V$ is an orthonormal matrix, whose columns are called right singular vectors. + +For large matrices, usually we don't need the complete factorization but only the top singular +values and its associated singular vectors. This can save storage, and more importantly, de-noise +and recover the low-rank structure of the matrix. + +If we keep the top $k$ singular values, then the dimensions of the return will be: + +* `$U$`: `$m \times k$`, +* `$\Sigma$`: `$k \times k$`, +* `$V$`: `$n \times k$`. + +In this release, we provide SVD computation to row-oriented matrices that have only a few columns, +say, less than $1000$, but many rows, which we call *tall-and-skinny*. + +
      +
      +{% highlight scala %} +val mat: RowMatrix = ... + +// Compute the top 20 singular values and corresponding singular vectors. +val svd: SingularValueDecomposition[RowMatrix, Matrix] = mat.computeSVD(20, computeU = true) +val U: RowMatrix = svd.U // The U factor is a RowMatrix. +val s: Vector = svd.s // The singular values are stored in a local dense vector. +val V: Matrix = svd.V // The V factor is a local dense matrix. +{% endhighlight %} +
      +Same code applies to `IndexedRowMatrix`. +The only difference that the `U` matrix becomes an `IndexedRowMatrix`. +
      + +## Principal component analysis (PCA) + +[Principal component analysis (PCA)](http://en.wikipedia.org/wiki/Principal_component_analysis) is a +statistical method to find a rotation such that the first coordinate has the largest variance +possible, and each succeeding coordinate in turn has the largest variance possible. The columns of +the rotation matrix are called principal components. PCA is used widely in dimensionality reduction. + +In this release, we implement PCA for tall-and-skinny matrices stored in row-oriented format. + +
      +
      + +The following code demonstrates how to compute principal components on a tall-and-skinny `RowMatrix` +and use them to project the vectors into a low-dimensional space. +The number of columns should be small, e.g, less than 1000. + +{% highlight scala %} +val mat: RowMatrix = ... + +// Compute the top 10 principal components. +val pc: Matrix = mat.computePrincipalComponents(10) // Principal components are stored in a local dense matrix. + +// Project the rows to the linear space spanned by the top 10 principal components. +val projected: RowMatrix = mat.multiply(pc) +{% endhighlight %} +
      +
      diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 0963a99881c9d..c49f857d07557 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -3,63 +3,121 @@ layout: global title: Machine Learning Library (MLlib) --- +MLlib is a Spark implementation of some common machine learning algorithms and utilities, +including classification, regression, clustering, collaborative +filtering, dimensionality reduction, as well as underlying optimization primitives: -MLlib is a Spark implementation of some common machine learning (ML) -functionality, as well associated tests and data generators. MLlib -currently supports four common types of machine learning problem settings, -namely classification, regression, clustering and collaborative filtering, -as well as an underlying gradient descent optimization primitive and several -linear algebra methods. - -# Available Methods -The following links provide a detailed explanation of the methods and usage examples for each of them: - -* Classification and Regression - * Binary Classification - * SVM (L1 and L2 regularized) - * Logistic Regression (L1 and L2 regularized) - * Linear Regression - * Least Squares - * Lasso - * Ridge Regression - * Decision Tree (for classification and regression) -* Clustering - * k-Means -* Collaborative Filtering - * Matrix Factorization using Alternating Least Squares -* Optimization - * Gradient Descent and Stochastic Gradient Descent -* Linear Algebra - * Singular Value Decomposition - * Principal Component Analysis - -# Data Types - -Most MLlib algorithms operate on RDDs containing vectors. In Java and Scala, the -[Vector](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) class is used to -represent vectors. You can create either dense or sparse vectors using the -[Vectors](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) factory. - -In Python, MLlib can take the following vector types: - -* [NumPy](http://www.numpy.org) arrays -* Standard Python lists (e.g. `[1, 2, 3]`) -* The MLlib [SparseVector](api/python/pyspark.mllib.linalg.SparseVector-class.html) class -* [SciPy sparse matrices](http://docs.scipy.org/doc/scipy/reference/sparse.html) - -For efficiency, we recommend using NumPy arrays over lists, and using the -[CSC format](http://docs.scipy.org/doc/scipy/reference/generated/scipy.sparse.csc_matrix.html#scipy.sparse.csc_matrix) -for SciPy matrices, or MLlib's own SparseVector class. - -Several other simple data types are used throughout the library, e.g. the LabeledPoint -class ([Java/Scala](api/scala/index.html#org.apache.spark.mllib.regression.LabeledPoint), -[Python](api/python/pyspark.mllib.regression.LabeledPoint-class.html)) for labeled data. - -# Dependencies -MLlib uses the [jblas](https://github.com/mikiobraun/jblas) linear algebra library, which itself -depends on native Fortran routines. You may need to install the -[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) -if it is not already present on your nodes. MLlib will throw a linking error if it cannot -detect these libraries automatically. +* [Basics](mllib-basics.html) + * data types + * summary statistics +* Classification and regression + * [linear support vector machine (SVM)](mllib-linear-methods.html#linear-support-vector-machine-svm) + * [logistic regression](mllib-linear-methods.html#logistic-regression) + * [linear least squares, Lasso, and ridge regression](mllib-linear-methods.html#linear-least-squares-lasso-and-ridge-regression) + * [decision tree](mllib-decision-tree.html) + * [naive Bayes](mllib-naive-bayes.html) +* [Collaborative filtering](mllib-collaborative-filtering.html) + * alternating least squares (ALS) +* [Clustering](mllib-clustering.html) + * k-means +* [Dimensionality reduction](mllib-dimensionality-reduction.html) + * singular value decomposition (SVD) + * principal component analysis (PCA) +* [Optimization](mllib-optimization.html) + * stochastic gradient descent + * limited-memory BFGS (L-BFGS) + +MLlib is currently a *beta* component under active development. +The APIs may change in the future releases, and we will provide migration guide between releases. + +## Dependencies + +MLlib uses linear algebra packages [Breeze](http://www.scalanlp.org/), which depends on +[netlib-java](https://github.com/fommil/netlib-java), and +[jblas](https://github.com/mikiobraun/jblas). +`netlib-java` and `jblas` depend on native Fortran routines. +You need to install the +[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) if it is not +already present on your nodes. MLlib will throw a linking error if it cannot detect these libraries +automatically. Due to license issues, we do not include `netlib-java`'s native libraries in MLlib's +dependency set. If no native library is available at runtime, you will see a warning message. To +use native libraries from `netlib-java`, please include artifact +`com.github.fommil.netlib:all:1.1.2` as a dependency of your project or build your own (see +[instructions](https://github.com/fommil/netlib-java/blob/master/README.md#machine-optimised-system-libraries)). To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. + +--- + +## Migration guide + +### From 0.9 to 1.0 + +In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few +breaking changes. If your data is sparse, please store it in a sparse format instead of dense to +take advantage of sparsity in both storage and computation. + +
      +
      + +We used to represent a feature vector by `Array[Double]`, which is replaced by +[`Vector`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) in v1.0. Algorithms that used +to accept `RDD[Array[Double]]` now take +`RDD[Vector]`. [`LabeledPoint`](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint) +is now a wrapper of `(Double, Vector)` instead of `(Double, Array[Double])`. Converting +`Array[Double]` to `Vector` is straightforward: + +{% highlight scala %} +import org.apache.spark.mllib.linalg.{Vector, Vectors} + +val array: Array[Double] = ... // a double array +val vector: Vector = Vectors.dense(array) // a dense vector +{% endhighlight %} + +[`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to create sparse vectors. + +*Note*. Scala imports `scala.collection.immutable.Vector` by default, so you have to import `org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`. + +
      + +
      + +We used to represent a feature vector by `double[]`, which is replaced by +[`Vector`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) in v1.0. Algorithms that used +to accept `RDD` now take +`RDD`. [`LabeledPoint`](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint) +is now a wrapper of `(double, Vector)` instead of `(double, double[])`. Converting `double[]` to +`Vector` is straightforward: + +{% highlight java %} +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; + +double[] array = ... // a double array +Vector vector = Vectors.dense(array) // a dense vector +{% endhighlight %} + +[`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to +create sparse vectors. + +
      + +
      + +We used to represent a labeled feature vector in a NumPy array, where the first entry corresponds to +the label and the rest are features. This representation is replaced by class +[`LabeledPoint`](api/pyspark/pyspark.mllib.regression.LabeledPoint-class.html), which takes both +dense and sparse feature vectors. + +{% highlight python %} +from pyspark.mllib.linalg import SparseVector +from pyspark.mllib.regression import LabeledPoint + +# Create a labeled point with a positive label and a dense feature vector. +pos = LabeledPoint(1.0, [1.0, 0.0, 3.0]) + +# Create a labeled point with a negative label and a sparse feature vector. +neg = LabeledPoint(0.0, SparseVector(3, [0, 2], [1.0, 3.0])) +{% endhighlight %} +
      +
      diff --git a/docs/mllib-linear-algebra.md b/docs/mllib-linear-algebra.md deleted file mode 100644 index 09598be7903ac..0000000000000 --- a/docs/mllib-linear-algebra.md +++ /dev/null @@ -1,74 +0,0 @@ ---- -layout: global -title: MLlib - Linear Algebra ---- - -* Table of contents -{:toc} - - -# Singular Value Decomposition -Singular Value `Decomposition` for Tall and Skinny matrices. -Given an `$m \times n$` matrix `$A$`, we can compute matrices `$U,S,V$` such that - -`\[ - A = U \cdot S \cdot V^T - \]` - -There is no restriction on m, but we require n^2 doubles to -fit in memory locally on one machine. -Further, n should be less than m. - -The decomposition is computed by first computing `$A^TA = V S^2 V^T$`, -computing SVD locally on that (since `$n \times n$` is small), -from which we recover `$S$` and `$V$`. -Then we compute U via easy matrix multiplication -as `$U = A \cdot V \cdot S^{-1}$`. - -Only singular vectors associated with largest k singular values -are recovered. If there are k -such values, then the dimensions of the return will be: - -* `$S$` is `$k \times k$` and diagonal, holding the singular values on diagonal. -* `$U$` is `$m \times k$` and satisfies `$U^T U = \mathop{eye}(k)$`. -* `$V$` is `$n \times k$` and satisfies `$V^T V = \mathop{eye}(k)$`. - -All input and output is expected in sparse matrix format, 0-indexed -as tuples of the form ((i,j),value) all in -SparseMatrix RDDs. Below is example usage. - -{% highlight scala %} - -import org.apache.spark.SparkContext -import org.apache.spark.mllib.linalg.SVD -import org.apache.spark.mllib.linalg.SparseMatrix -import org.apache.spark.mllib.linalg.MatrixEntry - -// Load and parse the data file -val data = sc.textFile("mllib/data/als/test.data").map { line => - val parts = line.split(',') - MatrixEntry(parts(0).toInt, parts(1).toInt, parts(2).toDouble) -} -val m = 4 -val n = 4 -val k = 1 - -// recover largest singular vector -val decomposed = SVD.sparseSVD(SparseMatrix(data, m, n), k) -val = decomposed.S.data - -println("singular values = " + s.toArray.mkString) -{% endhighlight %} - - -# Principal Component Analysis - -Computes the top k principal component coefficients for the m-by-n data matrix X. -Rows of X correspond to observations and columns correspond to variables. -The coefficient matrix is n-by-k. Each column of the return matrix contains coefficients -for one principal component, and the columns are in descending -order of component variance. This function centers the data and uses the -singular value decomposition (SVD) algorithm. - -All input and output is expected in DenseMatrix matrix format. See the examples directory -under "SparkPCA.scala" for example usage. diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md new file mode 100644 index 0000000000000..8108440698495 --- /dev/null +++ b/docs/mllib-linear-methods.md @@ -0,0 +1,389 @@ +--- +layout: global +title: MLlib - Linear Methods +--- + +* Table of contents +{:toc} + + +`\[ +\newcommand{\R}{\mathbb{R}} +\newcommand{\E}{\mathbb{E}} +\newcommand{\x}{\mathbf{x}} +\newcommand{\y}{\mathbf{y}} +\newcommand{\wv}{\mathbf{w}} +\newcommand{\av}{\mathbf{\alpha}} +\newcommand{\bv}{\mathbf{b}} +\newcommand{\N}{\mathbb{N}} +\newcommand{\id}{\mathbf{I}} +\newcommand{\ind}{\mathbf{1}} +\newcommand{\0}{\mathbf{0}} +\newcommand{\unit}{\mathbf{e}} +\newcommand{\one}{\mathbf{1}} +\newcommand{\zero}{\mathbf{0}} +\]` + +## Mathematical formulation + +Many standard *machine learning* methods can be formulated as a convex optimization problem, i.e. +the task of finding a minimizer of a convex function `$f$` that depends on a variable vector +`$\wv$` (called `weights` in the code), which has `$d$` entries. +Formally, we can write this as the optimization problem `$\min_{\wv \in\R^d} \; f(\wv)$`, where +the objective function is of the form +`\begin{equation} + f(\wv) := + \frac1n \sum_{i=1}^n L(\wv;\x_i,y_i) + + \lambda\, R(\wv_i) + \label{eq:regPrimal} + \ . +\end{equation}` +Here the vectors `$\x_i\in\R^d$` are the training data examples, for `$1\le i\le n$`, and +`$y_i\in\R$` are their corresponding labels, which we want to predict. +We call the method *linear* if $L(\wv; \x, y)$ can be expressed as a function of $\wv^T x$ and $y$. +Several MLlib's classification and regression algorithms fall into this category, +and are discussed here. + +The objective function `$f$` has two parts: +the loss that measures the error of the model on the training data, +and the regularizer that measures the complexity of the model. +The loss function `$L(\wv;.)$` must be a convex function in `$\wv$`. +The fixed regularization parameter `$\lambda \ge 0$` (`regParam` in the code) defines the trade-off +between the two goals of small loss and small model complexity. + +### Loss functions + +The following table summarizes the loss functions and their gradients or sub-gradients for the +methods MLlib supports: + + + + + + + + + + + + + + + + + + + +
      loss function $L(\wv; \x, y)$gradient or sub-gradient
      hinge loss$\max \{0, 1-y \wv^T \x \}, \quad y \in \{-1, +1\}$$\begin{cases}-y \cdot \x & \text{if $y \wv^T \x <1$}, \\ 0 & +\text{otherwise}.\end{cases}$
      logistic loss$\log(1+\exp( -y \wv^T \x)), \quad y \in \{-1, +1\}$$-y \left(1-\frac1{1+\exp(-y \wv^T \x)} \right) \cdot \x$
      squared loss$\frac{1}{2} (\wv^T \x - y)^2, \quad y \in \R$$(\wv^T \x - y) \cdot \x$
      + +### Regularizers + +The purpose of the [regularizer](http://en.wikipedia.org/wiki/Regularization_(mathematics)) is to +encourage simple models, by punishing the complexity of the model `$\wv$`, in order to e.g. avoid +over-fitting. +We support the following regularizers in MLlib: + + + + + + + + + + + + + + + + +
      regularizer $R(\wv)$gradient or sub-gradient
      zero (unregularized)0$\0$
      L2$\frac{1}{2}\|\wv\|_2^2$$\wv$
      L1$\|\wv\|_1$$\mathrm{sign}(\wv)$
      + +Here `$\mathrm{sign}(\wv)$` is the vector consisting of the signs (`$\pm1$`) of all the entries +of `$\wv$`. + +L2-regularized problems are generally easier to solve than L1-regularized due to smoothness. +However, L1 regularization can help promote sparsity in weights, leading to simpler models, which is +also used for feature selection. It is not recommended to train models without any regularization, +especially when the number of training examples is small. + +## Binary classification + +[Binary classification](http://en.wikipedia.org/wiki/Binary_classification) is to divide items into +two categories: positive and negative. MLlib supports two linear methods for binary classification: +linear support vector machine (SVM) and logistic regression. The training data set is represented +by an RDD of [LabeledPoint](mllib-data-types.html) in MLlib. Note that, in the mathematical +formulation, a training label $y$ is either $+1$ (positive) or $-1$ (negative), which is convenient +for the formulation. *However*, the negative label is represented by $0$ in MLlib instead of $-1$, +to be consistent with multiclass labeling. + +### Linear support vector machine (SVM) + +The [linear SVM](http://en.wikipedia.org/wiki/Support_vector_machine#Linear_SVM) +has become a standard choice for large-scale classification tasks. +The name "linear SVM" is actually ambiguous. +By "linear SVM", we mean specifically the linear method with the loss function in formulation +`$\eqref{eq:regPrimal}$` given by the hinge loss +`\[ +L(\wv;\x,y) := \max \{0, 1-y \wv^T \x \}. +\]` +By default, linear SVMs are trained with an L2 regularization. +We also support alternative L1 regularization. In this case, +the problem becomes a [linear program](http://en.wikipedia.org/wiki/Linear_programming). + +Linear SVM algorithm outputs a SVM model, which makes predictions based on the value of $\wv^T \x$. +By the default, if $\wv^T \x \geq 0$, the outcome is positive, or negative otherwise. +However, quite often in practice, the default threshold $0$ is not a good choice. +The threshold should be determined via model evaluation. + +### Logistic regression + +[Logistic regression](http://en.wikipedia.org/wiki/Logistic_regression) is widely used to predict a +binary response. It is a linear method with the loss function in formulation +`$\eqref{eq:regPrimal}$` given by the logistic loss +`\[ +L(\wv;\x,y) := \log(1+\exp( -y \wv^T \x)). +\]` + +Logistic regression algorithm outputs a logistic regression model, which makes predictions by +applying the logistic function +`\[ +\mathrm{logit}(z) = \frac{1}{1 + e^{-z}} +\]` +$\wv^T \x$. +By default, if $\mathrm{logit}(\wv^T x) > 0.5$, the outcome is positive, or negative otherwise. +For the same reason mentioned above, quite often in practice, this default threshold is not a good choice. +The threshold should be determined via model evaluation. + +### Evaluation metrics + +MLlib supports common evaluation metrics for binary classification (not available in Python). This +includes precision, recall, [F-measure](http://en.wikipedia.org/wiki/F1_score), +[receiver operating characteristic (ROC)](http://en.wikipedia.org/wiki/Receiver_operating_characteristic), +precision-recall curve, and +[area under the curves (AUC)](http://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve). +Among the metrics, area under ROC is commonly used to compare models and precision/recall/F-measure +can help determine the threshold to use. + +### Examples + +
      + +
      +The following code snippet illustrates how to load a sample dataset, execute a +training algorithm on this training data using a static method in the algorithm +object, and make predictions with the resulting model to compute the training +error. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.classification.SVMWithSGD +import org.apache.spark.mllib.evaluation.binary.BinaryClassificationMetrics +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLUtils + +// Load training data in LIBSVM format. +val data = MLUtils.loadLibSVMData(sc, "mllib/data/sample_libsvm_data.txt") + +// Split data into training (60%) and test (40%). +val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) +val training = splits(0).cache() +val test = splits(1) + +// Run training algorithm to build the model +val numIterations = 100 +val model = SVMWithSGD.train(training, numIterations) + +// Clear the default threshold. +model.clearThreshold() + +// Compute raw scores on the test set. +val scoreAndLabels = test.map { point => + val score = model.predict(point.features) + (score, point.label) +} + +// Get evaluation metrics. +val metrics = new BinaryClassificationMetrics(scoreAndLabels) +val auROC = metrics.areaUnderROC() + +println("Area under ROC = " + auROC) +{% endhighlight %} + +The `SVMWithSGD.train()` method by default performs L2 regularization with the +regularization parameter set to 1.0. If we want to configure this algorithm, we +can customize `SVMWithSGD` further by creating a new object directly and +calling setter methods. All other MLlib algorithms support customization in +this way as well. For example, the following code produces an L1 regularized +variant of SVMs with regularization parameter set to 0.1, and runs the training +algorithm for 200 iterations. + +{% highlight scala %} +import org.apache.spark.mllib.optimization.L1Updater + +val svmAlg = new SVMWithSGD() +svmAlg.optimizer.setNumIterations(200) + .setRegParam(0.1) + .setUpdater(new L1Updater) +val modelL1 = svmAlg.run(parsedData) +{% endhighlight %} + +Similarly, you can use replace `SVMWithSGD` by +[`LogisticRegressionWithSGD`](api/mllib/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD). + +
      + +
      +All of MLlib's methods use Java-friendly types, so you can import and call them there the same +way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the +Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by +calling `.rdd()` on your `JavaRDD` object. +
      + +
      +The following example shows how to load a sample dataset, build Logistic Regression model, +and make predictions with the resulting model to compute the training error. + +{% highlight python %} +from pyspark.mllib.classification import LogisticRegressionWithSGD +from pyspark.mllib.regression import LabeledPoint +from numpy import array + +# Load and parse the data +def parsePoint(line): + values = [float(x) for x in line.split(' ')] + return LabeledPoint(values[0], values[1:]) + +data = sc.textFile("mllib/data/sample_svm_data.txt") +parsedData = data.map(parsePoint) + +# Build the model +model = LogisticRegressionWithSGD.train(parsedData) + +# Evaluating the model on training data +labelsAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) +trainErr = labelsAndPreds.filter(lambda (v, p): v != p).count() / float(parsedData.count()) +print("Training Error = " + str(trainErr)) +{% endhighlight %} +
      +
      + +## Linear least squares, Lasso, and ridge regression + +Linear least squares is a family of linear methods with the loss function in formulation +`$\eqref{eq:regPrimal}$` given by the squared loss + +`\[ +L(\wv;\x,y) := \frac{1}{2} (\wv^T \x - y)^2. +\]` + +Depending on the regularization type, we call the method +[*ordinary least squares*](http://en.wikipedia.org/wiki/Ordinary_least_squares) or simply +[*linear least squares*](http://en.wikipedia.org/wiki/Linear_least_squares_(mathematics)) if there +is no regularization, [*ridge regression*](http://en.wikipedia.org/wiki/Ridge_regression) if L2 +regularization is used, and [*Lasso*](http://en.wikipedia.org/wiki/Lasso_(statistics)) if L1 +regularization is used. This average loss $\frac{1}{n} \sum_{i=1}^n (\wv^T x_i - y_i)^2$ is also +known as the [mean squared error](http://en.wikipedia.org/wiki/Mean_squared_error). + +Note that the squared loss is sensitive to outliers. +Regularization or a robust alternative (e.g., $\ell_1$ regression) is usually necessary in practice. + +### Examples + +
      + +
      +The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. +The example then uses LinearRegressionWithSGD to build a simple linear model to predict label +values. We compute the Mean Squared Error at the end to evaluate +[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). + +{% highlight scala %} +import org.apache.spark.mllib.regression.LinearRegressionWithSGD +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.Vectors + +// Load and parse the data +val data = sc.textFile("mllib/data/ridge-data/lpsa.data") +val parsedData = data.map { line => + val parts = line.split(',') + LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) +} + +// Building the model +val numIterations = 100 +val model = LinearRegressionWithSGD.train(parsedData, numIterations) + +// Evaluate model on training examples and compute training error +val valuesAndPreds = parsedData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.reduce(_ + _) / valuesAndPreds.count +println("training Mean Squared Error = " + MSE) +{% endhighlight %} + +Similarly you can use +[`RidgeRegressionWithSGD`](api/mllib/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) +and [`LassoWithSGD`](api/mllib/index.html#org.apache.spark.mllib.regression.LassoWithSGD). + +
      + +
      +All of MLlib's methods use Java-friendly types, so you can import and call them there the same +way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the +Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by +calling `.rdd()` on your `JavaRDD` object. +
      + +
      +The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. +The example then uses LinearRegressionWithSGD to build a simple linear model to predict label +values. We compute the Mean Squared Error at the end to evaluate +[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). + +{% highlight python %} +from pyspark.mllib.regression import LabeledPoint, LinearRegressionWithSGD +from numpy import array + +# Load and parse the data +def parsePoint(line): + values = [float(x) for x in line.replace(',', ' ').split(' ')] + return LabeledPoint(values[0], values[1:]) + +data = sc.textFile("mllib/data/ridge-data/lpsa.data") +parsedData = data.map(parsePoint) + +# Build the model +model = LinearRegressionWithSGD.train(parsedData) + +# Evaluate the model on training data +valuesAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) +MSE = valuesAndPreds.map(lambda (v, p): (v - p)**2).reduce(lambda x, y: x + y) / valuesAndPreds.count() +print("Mean Squared Error = " + str(MSE)) +{% endhighlight %} +
      +
      + +## Implementation (developer) + +Behind the scene, MLlib implements a simple distributed version of stochastic gradient descent +(SGD), building on the underlying gradient descent primitive (as described in the optimization section). All provided algorithms take as input a +regularization parameter (`regParam`) along with various parameters associated with stochastic +gradient descent (`stepSize`, `numIterations`, `miniBatchFraction`). For each of them, we support +all three possible regularizations (none, L1 or L2). + +Algorithms are all implemented in Scala: + +* [SVMWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.SVMWithSGD) +* [LogisticRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) +* [LinearRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD) +* [RidgeRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) +* [LassoWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.LassoWithSGD) + +Python calls the Scala implementation via +[PythonMLLibAPI](api/mllib/index.html#org.apache.spark.mllib.api.python.PythonMLLibAPI). diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md new file mode 100644 index 0000000000000..6160fe5b2fe8c --- /dev/null +++ b/docs/mllib-naive-bayes.md @@ -0,0 +1,115 @@ +--- +layout: global +title: MLlib - Naive Bayes +--- + +Naive Bayes is a simple multiclass classification algorithm with the assumption of independence +between every pair of features. Naive Bayes can be trained very efficiently. Within a single pass to +the training data, it computes the conditional probability distribution of each feature given label, +and then it applies Bayes' theorem to compute the conditional probability distribution of label +given an observation and use it for prediction. For more details, please visit the wikipedia page +[Naive Bayes classifier](http://en.wikipedia.org/wiki/Naive_Bayes_classifier). + +In MLlib, we implemented multinomial naive Bayes, which is typically used for document +classification. Within that context, each observation is a document, each feature represents a term, +whose value is the frequency of the term. For its formulation, please visit the wikipedia page +[Multinomial naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes) +or the section +[Naive Bayes text classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html) +from the book Introduction to Information +Retrieval. [Additive smoothing](http://en.wikipedia.org/wiki/Lidstone_smoothing) can be used by +setting the parameter $\lambda$ (default to $1.0$). For document classification, the input feature +vectors are usually sparse. Please supply sparse vectors as input to take advantage of +sparsity. Since the training data is only used once, it is not necessary to cache it. + +## Examples + +
      +
      + +[NaiveBayes](api/mllib/index.html#org.apache.spark.mllib.classification.NaiveBayes$) implements +multinomial naive Bayes. It takes an RDD of +[LabeledPoint](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint) and an optional +smoothing parameter `lambda` as input, and output a +[NaiveBayesModel](api/mllib/index.html#org.apache.spark.mllib.classification.NaiveBayesModel), which +can be used for evaluation and prediction. + +{% highlight scala %} +import org.apache.spark.mllib.classification.NaiveBayes + +val training: RDD[LabeledPoint] = ... // training set +val test: RDD[LabeledPoint] = ... // test set + +val model = NaiveBayes.train(training, lambda = 1.0) +val prediction = model.predict(test.map(_.features)) + +val predictionAndLabel = prediction.zip(test.map(_.label)) +val accuracy = 1.0 * predictionAndLabel.filter(x => x._1 == x._2).count() / test.count() +{% endhighlight %} +
      + +
      + +[NaiveBayes](api/mllib/index.html#org.apache.spark.mllib.classification.NaiveBayes$) implements +multinomial naive Bayes. It takes a Scala RDD of +[LabeledPoint](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint) and an +optionally smoothing parameter `lambda` as input, and output a +[NaiveBayesModel](api/mllib/index.html#org.apache.spark.mllib.classification.NaiveBayesModel), which +can be used for evaluation and prediction. + +{% highlight java %} +import org.apache.spark.mllib.classification.NaiveBayes; + +JavaRDD training = ... // training set +JavaRDD test = ... // test set + +NaiveBayesModel model = NaiveBayes.train(training.rdd(), 1.0); + +JavaRDD prediction = model.predict(test.map(new Function() { + public Vector call(LabeledPoint p) { + return p.features(); + } + }) +JavaPairRDD predictionAndLabel = + prediction.zip(test.map(new Function() { + public Double call(LabeledPoint p) { + return p.label(); + } + }) +double accuracy = 1.0 * predictionAndLabel.filter(new Function, Boolean>() { + public Boolean call(Tuple2 pl) { + return pl._1() == pl._2(); + } + }).count() / test.count() +{% endhighlight %} +
      + +
      + +[NaiveBayes](api/pyspark/pyspark.mllib.classification.NaiveBayes-class.html) implements multinomial +naive Bayes. It takes an RDD of +[LabeledPoint](api/pyspark/pyspark.mllib.regression.LabeledPoint-class.html) and an optionally +smoothing parameter `lambda` as input, and output a +[NaiveBayesModel](api/pyspark/pyspark.mllib.classification.NaiveBayesModel-class.html), which can be +used for evaluation and prediction. + + +{% highlight python %} +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.classification import NaiveBayes + +# an RDD of LabeledPoint +data = sc.parallelize([ + LabeledPoint(0.0, [0.0, 0.0]) + ... # more labeled points +]) + +# Train a naive Bayes model. +model = NaiveBayes.train(data, 1.0) + +# Make prediction. +prediction = model.predict([0.0, 0.0]) +{% endhighlight %} + +
      +
      diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index c79cc3d944f32..bec3912b55dc7 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -1,6 +1,6 @@ --- layout: global -title: MLlib - Optimization +title: MLlib - Optimization --- * Table of contents @@ -25,9 +25,10 @@ title: MLlib - Optimization -# Mathematical Description +## Mathematical description + +### Gradient descent -## (Sub)Gradient Descent The simplest method to solve optimization problems of the form `$\min_{\wv \in\R^d} \; f(\wv)$` is [gradient descent](http://en.wikipedia.org/wiki/Gradient_descent). Such first-order optimization methods (including gradient descent and stochastic variants @@ -38,14 +39,14 @@ the direction of steepest descent, which is the negative of the derivative (call [gradient](http://en.wikipedia.org/wiki/Gradient)) of the function at the current point, i.e., at the current parameter value. If the objective function `$f$` is not differentiable at all arguments, but still convex, then a -*subgradient* +*sub-gradient* is the natural generalization of the gradient, and assumes the role of the step direction. -In any case, computing a gradient or subgradient of `$f$` is expensive --- it requires a full +In any case, computing a gradient or sub-gradient of `$f$` is expensive --- it requires a full pass through the complete dataset, in order to compute the contributions from all loss terms. -## Stochastic (Sub)Gradient Descent (SGD) +### Stochastic gradient descent (SGD) Optimization problems whose objective function `$f$` is written as a sum are particularly -suitable to be solved using *stochastic subgradient descent (SGD)*. +suitable to be solved using *stochastic gradient descent (SGD)*. In our case, for the optimization formulations commonly used in supervised machine learning, `\begin{equation} @@ -98,7 +99,7 @@ For the L1-regularizer, the proximal operator is given by soft thresholding, as [L1Updater](api/scala/index.html#org.apache.spark.mllib.optimization.L1Updater). -## Update Schemes for Distributed SGD +### Update schemes for distributed SGD The SGD implementation in [GradientDescent](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) uses a simple (distributed) sampling of the data examples. @@ -129,12 +130,12 @@ point. -# Implementation in MLlib +## Implementation in MLlib Gradient descent methods including stochastic subgradient descent (SGD) as included as a low-level primitive in `MLlib`, upon which various ML algorithms are developed, see the -classification and regression +linear methods section for example. The SGD method @@ -161,6 +162,4 @@ each iteration, to compute the gradient direction. Available algorithms for gradient descent: -* [GradientDescent.runMiniBatchSGD](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) - - +* [GradientDescent.runMiniBatchSGD](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) diff --git a/mllib/data/sample_libsvm_data.txt b/mllib/data/sample_libsvm_data.txt new file mode 100644 index 0000000000000..861c70cde7fd2 --- /dev/null +++ b/mllib/data/sample_libsvm_data.txt @@ -0,0 +1,100 @@ +0 128:51 129:159 130:253 131:159 132:50 155:48 156:238 157:252 158:252 159:252 160:237 182:54 183:227 184:253 185:252 186:239 187:233 188:252 189:57 190:6 208:10 209:60 210:224 211:252 212:253 213:252 214:202 215:84 216:252 217:253 218:122 236:163 237:252 238:252 239:252 240:253 241:252 242:252 243:96 244:189 245:253 246:167 263:51 264:238 265:253 266:253 267:190 268:114 269:253 270:228 271:47 272:79 273:255 274:168 290:48 291:238 292:252 293:252 294:179 295:12 296:75 297:121 298:21 301:253 302:243 303:50 317:38 318:165 319:253 320:233 321:208 322:84 329:253 330:252 331:165 344:7 345:178 346:252 347:240 348:71 349:19 350:28 357:253 358:252 359:195 372:57 373:252 374:252 375:63 385:253 386:252 387:195 400:198 401:253 402:190 413:255 414:253 415:196 427:76 428:246 429:252 430:112 441:253 442:252 443:148 455:85 456:252 457:230 458:25 467:7 468:135 469:253 470:186 471:12 483:85 484:252 485:223 494:7 495:131 496:252 497:225 498:71 511:85 512:252 513:145 521:48 522:165 523:252 524:173 539:86 540:253 541:225 548:114 549:238 550:253 551:162 567:85 568:252 569:249 570:146 571:48 572:29 573:85 574:178 575:225 576:253 577:223 578:167 579:56 595:85 596:252 597:252 598:252 599:229 600:215 601:252 602:252 603:252 604:196 605:130 623:28 624:199 625:252 626:252 627:253 628:252 629:252 630:233 631:145 652:25 653:128 654:252 655:253 656:252 657:141 658:37 +1 159:124 160:253 161:255 162:63 186:96 187:244 188:251 189:253 190:62 214:127 215:251 216:251 217:253 218:62 241:68 242:236 243:251 244:211 245:31 246:8 268:60 269:228 270:251 271:251 272:94 296:155 297:253 298:253 299:189 323:20 324:253 325:251 326:235 327:66 350:32 351:205 352:253 353:251 354:126 378:104 379:251 380:253 381:184 382:15 405:80 406:240 407:251 408:193 409:23 432:32 433:253 434:253 435:253 436:159 460:151 461:251 462:251 463:251 464:39 487:48 488:221 489:251 490:251 491:172 515:234 516:251 517:251 518:196 519:12 543:253 544:251 545:251 546:89 570:159 571:255 572:253 573:253 574:31 597:48 598:228 599:253 600:247 601:140 602:8 625:64 626:251 627:253 628:220 653:64 654:251 655:253 656:220 681:24 682:193 683:253 684:220 +1 125:145 126:255 127:211 128:31 152:32 153:237 154:253 155:252 156:71 180:11 181:175 182:253 183:252 184:71 209:144 210:253 211:252 212:71 236:16 237:191 238:253 239:252 240:71 264:26 265:221 266:253 267:252 268:124 269:31 293:125 294:253 295:252 296:252 297:108 322:253 323:252 324:252 325:108 350:255 351:253 352:253 353:108 378:253 379:252 380:252 381:108 406:253 407:252 408:252 409:108 434:253 435:252 436:252 437:108 462:255 463:253 464:253 465:170 490:253 491:252 492:252 493:252 494:42 518:149 519:252 520:252 521:252 522:144 546:109 547:252 548:252 549:252 550:144 575:218 576:253 577:253 578:255 579:35 603:175 604:252 605:252 606:253 607:35 631:73 632:252 633:252 634:253 635:35 659:31 660:211 661:252 662:253 663:35 +1 153:5 154:63 155:197 181:20 182:254 183:230 184:24 209:20 210:254 211:254 212:48 237:20 238:254 239:255 240:48 265:20 266:254 267:254 268:57 293:20 294:254 295:254 296:108 321:16 322:239 323:254 324:143 350:178 351:254 352:143 378:178 379:254 380:143 406:178 407:254 408:162 434:178 435:254 436:240 462:113 463:254 464:240 490:83 491:254 492:245 493:31 518:79 519:254 520:246 521:38 547:214 548:254 549:150 575:144 576:241 577:8 603:144 604:240 605:2 631:144 632:254 633:82 659:230 660:247 661:40 687:168 688:209 689:31 +1 152:1 153:168 154:242 155:28 180:10 181:228 182:254 183:100 209:190 210:254 211:122 237:83 238:254 239:162 265:29 266:254 267:248 268:25 293:29 294:255 295:254 296:103 321:29 322:254 323:254 324:109 349:29 350:254 351:254 352:109 377:29 378:254 379:254 380:109 405:29 406:255 407:254 408:109 433:29 434:254 435:254 436:109 461:29 462:254 463:254 464:63 489:29 490:254 491:254 492:28 517:29 518:254 519:254 520:28 545:29 546:254 547:254 548:35 573:29 574:254 575:254 576:109 601:6 602:212 603:254 604:109 630:203 631:254 632:178 658:155 659:254 660:190 686:32 687:199 688:104 +0 130:64 131:253 132:255 133:63 157:96 158:205 159:251 160:253 161:205 162:111 163:4 184:96 185:189 186:251 187:251 188:253 189:251 190:251 191:31 209:16 210:64 211:223 212:244 213:251 214:251 215:211 216:213 217:251 218:251 219:31 236:80 237:181 238:251 239:253 240:251 241:251 242:251 243:94 244:96 245:251 246:251 247:31 263:92 264:253 265:253 266:253 267:255 268:253 269:253 270:253 271:95 272:96 273:253 274:253 275:31 290:92 291:236 292:251 293:243 294:220 295:233 296:251 297:251 298:243 299:82 300:96 301:251 302:251 303:31 317:80 318:253 319:251 320:251 321:188 323:96 324:251 325:251 326:109 328:96 329:251 330:251 331:31 344:96 345:240 346:253 347:243 348:188 349:42 351:96 352:204 353:109 354:4 356:12 357:197 358:251 359:31 372:221 373:251 374:253 375:121 379:36 380:23 385:190 386:251 387:31 399:48 400:234 401:253 413:191 414:253 415:31 426:44 427:221 428:251 429:251 440:12 441:197 442:251 443:31 454:190 455:251 456:251 457:251 468:96 469:251 470:251 471:31 482:190 483:251 484:251 485:113 495:40 496:234 497:251 498:219 499:23 510:190 511:251 512:251 513:94 522:40 523:217 524:253 525:231 526:47 538:191 539:253 540:253 541:253 548:12 549:174 550:253 551:253 552:219 553:39 566:67 567:236 568:251 569:251 570:191 571:190 572:111 573:72 574:190 575:191 576:197 577:251 578:243 579:121 580:39 595:63 596:236 597:251 598:253 599:251 600:251 601:251 602:251 603:253 604:251 605:188 606:94 624:27 625:129 626:253 627:251 628:251 629:251 630:251 631:229 632:168 633:15 654:95 655:212 656:251 657:211 658:94 659:59 +1 159:121 160:254 161:136 186:13 187:230 188:253 189:248 190:99 213:4 214:118 215:253 216:253 217:225 218:42 241:61 242:253 243:253 244:253 245:74 268:32 269:206 270:253 271:253 272:186 273:9 296:211 297:253 298:253 299:239 300:69 324:254 325:253 326:253 327:133 351:142 352:255 353:253 354:186 355:8 378:149 379:229 380:254 381:207 382:21 405:54 406:229 407:253 408:254 409:105 433:152 434:254 435:254 436:213 437:26 460:112 461:251 462:253 463:253 464:26 487:29 488:212 489:253 490:250 491:149 514:36 515:214 516:253 517:253 518:137 542:75 543:253 544:253 545:253 546:59 570:93 571:253 572:253 573:189 574:17 598:224 599:253 600:253 601:84 625:43 626:235 627:253 628:126 629:1 653:99 654:248 655:253 656:119 682:225 683:235 684:49 +1 100:166 101:222 102:55 128:197 129:254 130:218 131:5 155:29 156:249 157:254 158:254 159:9 183:45 184:254 185:254 186:174 187:2 210:4 211:164 212:254 213:254 214:85 238:146 239:254 240:254 241:254 242:85 265:101 266:245 267:254 268:254 269:254 270:85 292:97 293:248 294:254 295:204 296:254 297:254 298:85 315:12 316:59 317:98 318:151 319:237 320:254 321:254 322:109 323:35 324:254 325:254 326:85 343:41 344:216 345:254 346:254 347:239 348:153 349:37 350:4 351:32 352:254 353:254 354:85 372:7 373:44 374:44 375:30 379:32 380:254 381:254 382:96 407:19 408:230 409:254 410:174 436:197 437:254 438:110 464:197 465:254 466:85 492:197 493:253 494:63 515:37 516:54 517:54 518:45 519:26 520:84 521:221 522:84 523:21 524:31 525:162 526:78 540:6 541:41 542:141 543:244 544:254 545:254 546:248 547:236 548:254 549:254 550:254 551:233 552:239 553:254 554:138 567:23 568:167 569:254 570:254 571:254 572:254 573:229 574:228 575:185 576:138 577:138 578:138 579:138 580:138 581:138 582:44 595:113 596:254 597:254 598:254 599:179 600:64 601:5 623:32 624:209 625:183 626:97 +0 155:53 156:255 157:253 158:253 159:253 160:124 183:180 184:253 185:251 186:251 187:251 188:251 189:145 190:62 209:32 210:217 211:241 212:253 213:251 214:251 215:251 216:251 217:253 218:107 237:37 238:251 239:251 240:253 241:251 242:251 243:251 244:251 245:253 246:107 265:166 266:251 267:251 268:253 269:251 270:96 271:148 272:251 273:253 274:107 291:73 292:253 293:253 294:253 295:253 296:130 299:110 300:253 301:255 302:108 319:73 320:251 321:251 322:251 323:251 327:109 328:251 329:253 330:107 347:202 348:251 349:251 350:251 351:225 354:6 355:129 356:251 357:253 358:107 375:150 376:251 377:251 378:251 379:71 382:115 383:251 384:251 385:253 386:107 403:253 404:251 405:251 406:173 407:20 410:217 411:251 412:251 413:253 414:107 430:182 431:255 432:253 433:216 438:218 439:253 440:253 441:182 457:63 458:221 459:253 460:251 461:215 465:84 466:236 467:251 468:251 469:77 485:109 486:251 487:253 488:251 489:215 492:11 493:160 494:251 495:251 496:96 513:109 514:251 515:253 516:251 517:137 520:150 521:251 522:251 523:251 524:71 541:109 542:251 543:253 544:251 545:35 547:130 548:253 549:251 550:251 551:173 552:20 569:110 570:253 571:255 572:253 573:98 574:150 575:253 576:255 577:253 578:164 597:109 598:251 599:253 600:251 601:251 602:251 603:251 604:253 605:251 606:35 625:93 626:241 627:253 628:251 629:251 630:251 631:251 632:216 633:112 634:5 654:103 655:253 656:251 657:251 658:251 659:251 683:124 684:251 685:225 686:71 687:71 +0 128:73 129:253 130:227 131:73 132:21 156:73 157:251 158:251 159:251 160:174 182:16 183:166 184:228 185:251 186:251 187:251 188:122 210:62 211:220 212:253 213:251 214:251 215:251 216:251 217:79 238:79 239:231 240:253 241:251 242:251 243:251 244:251 245:232 246:77 264:145 265:253 266:253 267:253 268:255 269:253 270:253 271:253 272:253 273:255 274:108 292:144 293:251 294:251 295:251 296:253 297:168 298:107 299:169 300:251 301:253 302:189 303:20 318:27 319:89 320:236 321:251 322:235 323:215 324:164 325:15 326:6 327:129 328:251 329:253 330:251 331:35 345:47 346:211 347:253 348:251 349:251 350:142 354:37 355:251 356:251 357:253 358:251 359:35 373:109 374:251 375:253 376:251 377:251 378:142 382:11 383:148 384:251 385:253 386:251 387:164 400:11 401:150 402:253 403:255 404:211 405:25 410:11 411:150 412:253 413:255 414:211 415:25 428:140 429:251 430:251 431:253 432:107 438:37 439:251 440:251 441:211 442:46 456:190 457:251 458:251 459:253 460:128 461:5 466:37 467:251 468:251 469:51 484:115 485:251 486:251 487:253 488:188 489:20 492:32 493:109 494:129 495:251 496:173 497:103 512:217 513:251 514:251 515:201 516:30 520:73 521:251 522:251 523:251 524:71 540:166 541:253 542:253 543:255 544:149 545:73 546:150 547:253 548:255 549:253 550:253 551:143 568:140 569:251 570:251 571:253 572:251 573:251 574:251 575:251 576:253 577:251 578:230 579:61 596:190 597:251 598:251 599:253 600:251 601:251 602:251 603:251 604:242 605:215 606:55 624:21 625:189 626:251 627:253 628:251 629:251 630:251 631:173 632:103 653:31 654:200 655:253 656:251 657:96 658:71 659:20 +1 155:178 156:255 157:105 182:6 183:188 184:253 185:216 186:14 210:14 211:202 212:253 213:253 214:23 238:12 239:199 240:253 241:128 242:6 266:42 267:253 268:253 269:158 294:42 295:253 296:253 297:158 322:155 323:253 324:253 325:158 350:160 351:253 352:253 353:147 378:160 379:253 380:253 381:41 405:17 406:225 407:253 408:235 409:31 433:24 434:253 435:253 436:176 461:24 462:253 463:253 464:176 489:24 490:253 491:253 492:176 517:24 518:253 519:253 520:176 545:24 546:253 547:253 548:162 573:46 574:253 575:253 576:59 601:142 602:253 603:253 604:59 629:142 630:253 631:253 632:59 657:142 658:253 659:202 660:8 685:87 686:253 687:139 +0 154:46 155:105 156:254 157:254 158:254 159:254 160:255 161:239 162:41 180:37 181:118 182:222 183:254 184:253 185:253 186:253 187:253 188:253 189:253 190:211 191:54 207:14 208:200 209:253 210:253 211:254 212:253 213:253 214:253 215:253 216:253 217:253 218:253 219:116 233:16 234:160 235:236 236:253 237:253 238:253 239:254 240:253 241:253 242:246 243:229 244:253 245:253 246:253 247:116 261:99 262:253 263:253 264:253 265:253 266:253 267:254 268:253 269:253 270:213 271:99 272:253 273:253 274:253 275:116 288:25 289:194 290:253 291:253 292:253 293:253 294:131 295:97 296:169 297:253 298:93 299:99 300:253 301:253 302:253 303:116 316:206 317:253 318:253 319:251 320:233 321:127 322:9 324:18 325:38 326:3 327:15 328:171 329:253 330:253 331:116 343:55 344:240 345:253 346:253 347:233 355:31 356:186 357:253 358:253 359:116 371:176 372:253 373:253 374:253 375:127 383:99 384:253 385:253 386:253 387:116 399:176 400:253 401:253 402:131 403:9 411:99 412:253 413:253 414:253 415:116 426:119 427:254 428:254 429:232 430:75 440:158 441:254 442:254 443:117 454:118 455:253 456:253 457:154 468:156 469:253 470:253 471:116 482:118 483:253 484:253 485:154 496:156 497:253 498:253 499:116 509:46 510:222 511:253 512:253 513:154 522:7 523:116 524:246 525:253 526:180 527:9 538:118 539:253 540:253 541:154 550:116 551:253 552:253 553:253 554:174 566:118 567:253 568:253 569:154 577:110 578:246 579:253 580:253 581:240 582:67 594:118 595:253 596:253 597:238 598:215 599:49 600:20 601:20 602:20 603:66 604:215 605:241 606:253 607:245 608:233 609:64 622:82 623:229 624:253 625:253 626:253 627:253 628:253 629:253 630:253 631:254 632:253 633:253 634:240 635:107 651:176 652:253 653:253 654:253 655:253 656:253 657:253 658:253 659:254 660:253 661:253 662:108 679:40 680:239 681:253 682:253 683:253 684:253 685:253 686:253 687:254 688:161 689:57 690:4 +0 152:56 153:105 154:220 155:254 156:63 178:18 179:166 180:233 181:253 182:253 183:253 184:236 185:209 186:209 187:209 188:77 189:18 206:84 207:253 208:253 209:253 210:253 211:253 212:254 213:253 214:253 215:253 216:253 217:172 218:8 233:57 234:238 235:253 236:253 237:253 238:253 239:253 240:254 241:253 242:253 243:253 244:253 245:253 246:119 260:14 261:238 262:253 263:253 264:253 265:253 266:253 267:253 268:179 269:196 270:253 271:253 272:253 273:253 274:238 275:12 288:33 289:253 290:253 291:253 292:253 293:253 294:248 295:134 297:18 298:83 299:237 300:253 301:253 302:253 303:14 316:164 317:253 318:253 319:253 320:253 321:253 322:128 327:57 328:119 329:214 330:253 331:94 343:57 344:248 345:253 346:253 347:253 348:126 349:14 350:4 357:179 358:253 359:248 360:56 371:175 372:253 373:253 374:240 375:190 376:28 385:179 386:253 387:253 388:173 399:209 400:253 401:253 402:178 413:92 414:253 415:253 416:208 427:211 428:254 429:254 430:179 442:135 443:255 444:209 455:209 456:253 457:253 458:90 470:134 471:253 472:208 483:209 484:253 485:253 486:178 497:2 498:142 499:253 500:208 511:209 512:253 513:253 514:214 515:35 525:30 526:253 527:253 528:208 539:165 540:253 541:253 542:253 543:215 544:36 553:163 554:253 555:253 556:164 567:18 568:172 569:253 570:253 571:253 572:214 573:127 574:7 580:72 581:232 582:253 583:171 584:17 596:8 597:182 598:253 599:253 600:253 601:253 602:162 603:56 607:64 608:240 609:253 610:253 611:14 625:7 626:173 627:253 628:253 629:253 630:253 631:245 632:241 633:239 634:239 635:246 636:253 637:225 638:14 639:1 654:18 655:59 656:138 657:224 658:253 659:253 660:254 661:253 662:253 663:253 664:240 665:96 685:37 686:104 687:192 688:255 689:253 690:253 691:182 692:73 +1 130:7 131:176 132:254 133:224 158:51 159:253 160:253 161:223 185:4 186:170 187:253 188:253 189:214 213:131 214:253 215:253 216:217 217:39 241:209 242:253 243:253 244:134 268:75 269:240 270:253 271:239 272:26 296:184 297:253 298:245 299:63 323:142 324:255 325:253 326:185 350:62 351:229 352:254 353:242 354:73 377:54 378:229 379:253 380:254 381:105 405:152 406:254 407:254 408:213 409:26 432:32 433:243 434:253 435:253 436:115 459:2 460:142 461:253 462:253 463:155 487:30 488:253 489:253 490:232 491:55 515:75 516:253 517:253 518:164 542:72 543:232 544:253 545:189 546:17 570:224 571:253 572:253 573:163 597:43 598:235 599:253 600:253 601:195 602:21 625:28 626:231 627:253 628:253 629:184 630:14 654:225 655:253 656:253 657:75 +0 155:21 156:176 157:253 158:253 159:124 182:105 183:176 184:251 185:251 186:251 187:251 188:105 208:58 209:217 210:241 211:253 212:251 213:251 214:251 215:251 216:243 217:113 218:5 235:63 236:231 237:251 238:251 239:253 240:251 241:251 242:251 243:251 244:253 245:251 246:113 263:144 264:251 265:251 266:251 267:253 268:251 269:251 270:251 271:251 272:253 273:251 274:215 290:125 291:253 292:253 293:253 294:253 295:255 296:253 297:253 298:253 299:253 300:255 301:253 302:227 303:42 318:253 319:251 320:251 321:251 322:251 323:253 324:251 325:251 326:251 327:251 328:253 329:251 330:251 331:142 345:27 346:253 347:251 348:251 349:235 350:241 351:253 352:251 353:246 354:137 355:35 356:98 357:251 358:251 359:236 360:61 372:47 373:211 374:253 375:251 376:235 377:82 378:103 379:253 380:251 381:137 384:73 385:251 386:251 387:251 388:71 399:27 400:211 401:251 402:253 403:251 404:86 407:72 408:71 409:10 412:73 413:251 414:251 415:173 416:20 427:89 428:253 429:253 430:255 431:253 432:35 440:73 441:253 442:253 443:253 444:72 454:84 455:236 456:251 457:251 458:253 459:251 460:138 468:73 469:251 470:251 471:251 472:71 481:63 482:236 483:251 484:251 485:251 486:227 487:251 488:246 489:138 490:11 494:16 495:37 496:228 497:251 498:246 499:137 500:10 509:73 510:251 511:251 512:251 513:173 514:42 515:142 516:142 517:142 518:41 522:109 523:251 524:253 525:251 526:137 537:73 538:251 539:251 540:173 541:20 549:27 550:211 551:251 552:253 553:147 554:10 565:73 566:253 567:253 568:143 575:21 576:176 577:253 578:253 579:253 593:73 594:251 595:251 596:205 597:144 603:176 604:251 605:251 606:188 607:107 621:62 622:236 623:251 624:251 625:251 626:218 627:217 628:217 629:217 630:217 631:253 632:230 633:189 634:20 650:83 651:158 652:251 653:251 654:253 655:251 656:251 657:251 658:251 659:253 660:107 679:37 680:251 681:251 682:253 683:251 684:251 685:251 686:122 687:72 688:30 +1 151:68 152:45 153:131 154:131 155:131 156:101 157:68 158:92 159:44 187:19 188:170 211:29 212:112 213:89 215:40 216:222 239:120 240:254 241:251 242:127 243:40 244:222 267:197 268:254 269:254 270:91 271:40 272:222 294:64 295:247 296:254 297:236 298:50 299:40 300:107 322:184 323:254 324:254 325:91 327:6 328:14 350:203 351:254 352:254 353:71 377:23 378:218 379:254 380:254 381:71 405:113 406:254 407:255 408:239 409:53 433:210 434:254 435:254 436:195 460:62 461:242 462:254 463:241 464:88 468:28 488:86 489:254 490:254 491:189 495:28 496:104 516:106 517:254 518:254 519:168 523:40 524:91 544:216 545:254 546:245 547:51 551:35 552:80 572:216 573:254 574:102 599:55 600:239 601:254 602:52 627:166 628:254 629:210 630:23 655:223 656:252 657:104 683:223 684:169 +0 125:29 126:170 127:255 128:255 129:141 151:29 152:198 153:255 154:255 155:255 156:226 157:255 158:86 178:141 179:255 180:255 181:170 182:29 184:86 185:255 186:255 187:141 204:29 205:226 206:255 207:198 208:57 213:226 214:255 215:255 216:226 217:114 231:29 232:255 233:255 234:114 241:141 242:170 243:114 244:255 245:255 246:141 259:226 260:255 261:170 269:29 270:57 273:141 274:255 275:226 286:57 287:255 288:170 302:114 303:255 304:198 314:226 315:255 331:170 332:255 333:57 342:255 343:226 360:255 361:170 370:255 371:170 388:114 389:198 398:255 399:226 416:86 417:255 426:198 427:255 444:86 445:255 454:114 455:255 456:57 472:86 473:255 482:29 483:255 484:226 500:141 501:255 511:170 512:255 513:170 528:226 529:198 539:29 540:226 541:255 542:170 555:29 556:255 557:114 568:29 569:226 570:255 571:141 582:57 583:226 584:226 598:141 599:255 600:255 601:170 602:86 607:29 608:86 609:226 610:255 611:226 612:29 627:86 628:198 629:255 630:255 631:255 632:255 633:255 634:255 635:255 636:255 637:255 638:141 639:29 657:29 658:114 659:170 660:170 661:170 662:170 663:170 664:86 +0 153:203 154:254 155:252 156:252 157:252 158:214 159:51 160:20 180:62 181:221 182:252 183:250 184:250 185:250 186:252 187:250 188:160 189:20 207:62 208:211 209:250 210:252 211:250 212:250 213:250 214:252 215:250 216:250 217:49 234:41 235:221 236:250 237:250 238:252 239:250 240:250 241:250 242:252 243:250 244:128 245:10 262:254 263:252 264:252 265:252 266:254 267:252 268:252 269:252 270:254 271:252 272:252 273:90 290:150 291:190 292:250 293:250 294:252 295:250 296:250 297:169 298:171 299:250 300:250 301:250 302:82 318:31 319:191 320:250 321:250 322:252 323:189 324:100 325:20 326:172 327:250 328:250 329:250 330:80 346:213 347:250 348:250 349:250 350:212 351:29 354:252 355:250 356:250 357:250 374:92 375:252 376:252 377:252 382:51 383:252 384:252 385:252 386:203 401:82 402:252 403:250 404:250 405:169 410:132 411:250 412:250 413:250 414:121 428:92 429:231 430:252 431:250 432:159 433:20 438:252 439:250 440:250 441:250 456:30 457:211 458:252 459:250 460:221 461:40 466:90 467:250 468:250 469:250 470:163 484:31 485:213 486:254 487:232 488:80 494:92 495:252 496:252 497:212 498:163 512:151 513:250 514:252 515:149 522:252 523:250 524:250 525:49 540:60 541:221 542:252 543:210 544:60 550:252 551:250 552:250 553:49 569:202 570:252 571:250 572:221 573:40 576:123 577:202 578:252 579:250 580:250 581:49 596:123 597:243 598:255 599:252 600:252 601:252 602:254 603:252 604:252 605:252 606:254 607:252 608:100 625:121 626:171 627:250 628:250 629:250 630:252 631:250 632:250 633:250 634:252 635:250 636:100 654:20 655:160 656:250 657:250 658:252 659:250 660:250 661:250 662:252 663:189 664:40 683:20 684:170 685:250 686:252 687:250 688:128 689:49 690:49 691:29 +1 98:64 99:191 100:70 125:68 126:243 127:253 128:249 129:63 152:30 153:223 154:253 155:253 156:247 157:41 179:73 180:238 181:253 182:253 183:253 184:242 206:73 207:236 208:253 209:253 210:253 211:253 212:242 234:182 235:253 236:253 237:191 238:247 239:253 240:149 262:141 263:253 264:143 265:86 266:249 267:253 268:122 290:9 291:36 292:7 293:14 294:233 295:253 296:122 322:230 323:253 324:122 350:230 351:253 352:122 378:231 379:255 380:123 406:230 407:253 408:52 433:61 434:245 435:253 461:98 462:253 463:253 468:35 469:12 489:98 490:253 491:253 494:9 495:142 496:233 497:146 517:190 518:253 519:253 520:128 521:7 522:99 523:253 524:253 525:180 544:29 545:230 546:253 547:253 548:252 549:210 550:253 551:253 552:253 553:140 571:28 572:207 573:253 574:253 575:253 576:254 577:253 578:253 579:235 580:70 581:9 599:126 600:253 601:253 602:253 603:253 604:254 605:253 606:168 607:19 627:79 628:253 629:253 630:201 631:190 632:132 633:63 634:5 +1 125:26 126:240 127:72 153:25 154:238 155:208 182:209 183:226 184:14 210:209 211:254 212:43 238:175 239:254 240:128 266:63 267:254 268:204 294:107 295:254 296:204 322:88 323:254 324:204 350:55 351:254 352:204 378:126 379:254 380:204 406:126 407:254 408:189 434:169 435:254 436:121 462:209 463:254 464:193 490:209 491:254 492:111 517:22 518:235 519:254 520:37 545:137 546:254 547:227 548:16 573:205 574:255 575:185 601:205 602:254 603:125 629:205 630:254 631:125 657:111 658:212 659:43 +0 155:62 156:91 157:213 158:255 159:228 160:91 161:12 182:70 183:230 184:253 185:253 186:253 187:253 188:253 189:152 190:7 210:246 211:253 212:253 213:253 214:253 215:253 216:253 217:253 218:106 237:21 238:247 239:253 240:253 241:253 242:253 243:253 244:253 245:208 246:24 265:156 266:253 267:253 268:253 269:253 270:253 271:253 272:253 273:195 292:88 293:238 294:253 295:253 296:253 297:221 298:253 299:253 300:253 301:195 320:230 321:253 322:253 323:253 324:198 325:40 326:177 327:253 328:253 329:195 346:56 347:156 348:251 349:253 350:189 351:182 352:15 354:86 355:240 356:253 357:210 358:28 374:213 375:253 376:253 377:156 378:3 383:205 384:253 385:253 386:106 401:121 402:252 403:253 404:135 405:3 411:46 412:253 413:253 414:106 428:28 429:212 430:253 431:248 432:23 439:42 440:253 441:253 442:106 456:197 457:253 458:234 459:70 467:42 468:253 469:253 470:106 483:11 484:202 485:253 486:187 495:58 496:253 497:210 498:27 511:107 512:253 513:253 514:40 522:53 523:227 524:253 525:195 539:107 540:253 541:253 542:40 549:47 550:227 551:253 552:231 553:58 567:107 568:253 569:253 570:40 575:5 576:131 577:222 578:253 579:231 580:59 595:14 596:204 597:253 598:226 599:222 600:73 601:58 602:58 603:170 604:253 605:253 606:227 607:58 624:197 625:253 626:253 627:253 628:253 629:253 630:253 631:253 632:253 633:238 634:58 652:33 653:179 654:241 655:253 656:253 657:253 658:253 659:250 660:116 661:14 682:75 683:179 684:253 685:151 686:89 687:86 +1 157:42 158:228 159:253 160:253 185:144 186:251 187:251 188:251 212:89 213:236 214:251 215:235 216:215 239:79 240:253 241:251 242:251 243:142 267:180 268:253 269:251 270:251 271:142 294:32 295:202 296:255 297:253 298:216 322:109 323:251 324:253 325:251 326:112 349:6 350:129 351:251 352:253 353:127 354:5 377:37 378:251 379:251 380:253 381:107 405:166 406:251 407:251 408:201 409:30 432:42 433:228 434:253 435:253 460:144 461:251 462:251 463:147 487:63 488:236 489:251 490:251 491:71 515:150 516:251 517:251 518:204 519:41 543:253 544:251 545:251 546:142 571:255 572:253 573:164 598:105 599:253 600:251 601:35 626:180 627:253 628:251 629:35 654:180 655:253 656:251 657:35 682:180 683:253 684:251 685:35 +1 128:62 129:254 130:213 156:102 157:253 158:252 159:102 160:20 184:102 185:254 186:253 187:254 188:50 212:102 213:253 214:252 215:253 216:50 240:102 241:254 242:253 243:254 244:50 268:142 269:253 270:252 271:253 272:50 295:51 296:253 297:254 298:253 299:224 300:20 323:132 324:252 325:253 326:252 327:162 351:173 352:253 353:254 354:253 355:102 378:82 379:253 380:252 381:253 382:252 383:61 406:203 407:254 408:253 409:254 410:233 433:41 434:243 435:253 436:252 437:253 438:111 461:132 462:253 463:254 464:253 465:203 488:41 489:253 490:252 491:253 492:252 493:40 515:11 516:213 517:254 518:253 519:254 520:151 543:92 544:252 545:253 546:252 547:192 548:50 570:21 571:214 572:253 573:255 574:253 575:41 598:142 599:253 600:252 601:253 602:171 625:113 626:253 627:255 628:253 629:203 630:40 653:30 654:131 655:233 656:111 +0 154:28 155:195 156:254 157:254 158:254 159:254 160:254 161:255 162:61 181:6 182:191 183:253 184:253 185:253 186:253 187:253 188:253 189:253 190:60 208:26 209:190 210:253 211:253 212:253 213:253 214:240 215:191 216:242 217:253 218:60 235:15 236:187 237:253 238:253 239:253 240:253 241:253 242:200 244:211 245:253 246:60 262:22 263:66 264:253 265:253 266:253 267:253 268:241 269:209 270:44 271:23 272:218 273:253 274:60 290:124 291:253 292:253 293:253 294:253 295:253 296:182 299:131 300:253 301:253 302:60 318:38 319:217 320:253 321:253 322:244 323:111 324:37 327:131 328:253 329:253 330:60 346:124 347:253 348:253 349:253 350:165 354:22 355:182 356:253 357:253 358:60 374:124 375:253 376:253 377:240 378:45 382:53 383:253 384:253 385:249 386:58 401:16 402:168 403:253 404:216 405:45 410:53 411:253 412:253 413:138 429:159 430:253 431:253 432:147 438:53 439:253 440:253 441:138 456:136 457:252 458:253 459:227 460:5 466:53 467:253 468:243 469:101 484:140 485:253 486:253 487:124 494:156 495:253 496:218 511:13 512:164 513:253 514:142 515:5 521:32 522:233 523:253 524:218 539:62 540:253 541:253 542:130 548:37 549:203 550:253 551:253 552:127 567:62 568:253 569:253 570:147 571:36 572:36 573:36 574:36 575:151 576:222 577:253 578:245 579:127 580:8 595:34 596:202 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:253 605:253 606:200 624:140 625:253 626:253 627:253 628:253 629:253 630:253 631:253 632:248 633:235 634:65 652:87 653:173 654:253 655:253 656:253 657:253 658:253 659:253 660:182 681:14 682:78 683:96 684:253 685:253 686:253 687:137 688:56 +0 123:8 124:76 125:202 126:254 127:255 128:163 129:37 130:2 150:13 151:182 152:253 153:253 154:253 155:253 156:253 157:253 158:23 177:15 178:179 179:253 180:253 181:212 182:91 183:218 184:253 185:253 186:179 187:109 205:105 206:253 207:253 208:160 209:35 210:156 211:253 212:253 213:253 214:253 215:250 216:113 232:19 233:212 234:253 235:253 236:88 237:121 238:253 239:233 240:128 241:91 242:245 243:253 244:248 245:114 260:104 261:253 262:253 263:110 264:2 265:142 266:253 267:90 270:26 271:199 272:253 273:248 274:63 287:1 288:173 289:253 290:253 291:29 293:84 294:228 295:39 299:72 300:251 301:253 302:215 303:29 315:36 316:253 317:253 318:203 319:13 328:82 329:253 330:253 331:170 343:36 344:253 345:253 346:164 356:11 357:198 358:253 359:184 360:6 371:36 372:253 373:253 374:82 385:138 386:253 387:253 388:35 399:128 400:253 401:253 402:47 413:48 414:253 415:253 416:35 427:154 428:253 429:253 430:47 441:48 442:253 443:253 444:35 455:102 456:253 457:253 458:99 469:48 470:253 471:253 472:35 483:36 484:253 485:253 486:164 496:16 497:208 498:253 499:211 500:17 511:32 512:244 513:253 514:175 515:4 524:44 525:253 526:253 527:156 540:171 541:253 542:253 543:29 551:30 552:217 553:253 554:188 555:19 568:171 569:253 570:253 571:59 578:60 579:217 580:253 581:253 582:70 596:78 597:253 598:253 599:231 600:48 604:26 605:128 606:249 607:253 608:244 609:94 610:15 624:8 625:151 626:253 627:253 628:234 629:101 630:121 631:219 632:229 633:253 634:253 635:201 636:80 653:38 654:232 655:253 656:253 657:253 658:253 659:253 660:253 661:253 662:201 663:66 +0 127:68 128:254 129:255 130:254 131:107 153:11 154:176 155:230 156:253 157:253 158:253 159:212 180:28 181:197 182:253 183:253 184:253 185:253 186:253 187:229 188:107 189:14 208:194 209:253 210:253 211:253 212:253 213:253 214:253 215:253 216:253 217:53 235:69 236:241 237:253 238:253 239:253 240:253 241:241 242:186 243:253 244:253 245:195 262:10 263:161 264:253 265:253 266:253 267:246 268:40 269:57 270:231 271:253 272:253 273:195 290:140 291:253 292:253 293:253 294:253 295:154 297:25 298:253 299:253 300:253 301:195 318:213 319:253 320:253 321:253 322:135 323:8 325:3 326:128 327:253 328:253 329:195 345:77 346:238 347:253 348:253 349:253 350:7 354:116 355:253 356:253 357:195 372:11 373:165 374:253 375:253 376:231 377:70 378:1 382:78 383:237 384:253 385:195 400:33 401:253 402:253 403:253 404:182 411:200 412:253 413:195 428:98 429:253 430:253 431:253 432:24 439:42 440:253 441:195 456:197 457:253 458:253 459:253 460:24 467:163 468:253 469:195 484:197 485:253 486:253 487:189 488:13 494:53 495:227 496:253 497:121 512:197 513:253 514:253 515:114 521:21 522:227 523:253 524:231 525:27 540:197 541:253 542:253 543:114 547:5 548:131 549:143 550:253 551:231 552:59 568:197 569:253 570:253 571:236 572:73 573:58 574:217 575:223 576:253 577:253 578:253 579:174 596:197 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:253 605:253 606:253 607:48 624:149 625:253 626:253 627:253 628:253 629:253 630:253 631:253 632:253 633:182 634:15 635:3 652:12 653:168 654:253 655:253 656:253 657:253 658:253 659:248 660:89 661:23 +1 157:85 158:255 159:103 160:1 185:205 186:253 187:253 188:30 213:205 214:253 215:253 216:30 240:44 241:233 242:253 243:244 244:27 268:135 269:253 270:253 271:100 296:153 297:253 298:240 299:76 323:12 324:208 325:253 326:166 351:69 352:253 353:253 354:142 378:14 379:110 380:253 381:235 382:33 406:63 407:223 408:235 409:130 434:186 435:253 436:235 437:37 461:17 462:145 463:253 464:231 465:35 489:69 490:220 491:231 492:123 516:18 517:205 518:253 519:176 520:27 543:17 544:125 545:253 546:185 547:39 571:71 572:214 573:231 574:41 599:167 600:253 601:225 602:33 626:72 627:205 628:207 629:14 653:30 654:249 655:233 656:49 681:32 682:253 683:89 +1 126:94 127:132 154:250 155:250 156:4 182:250 183:254 184:95 210:250 211:254 212:95 238:250 239:254 240:95 266:250 267:254 268:95 294:250 295:254 296:95 322:250 323:254 324:95 350:250 351:254 352:95 378:250 379:254 380:95 405:77 406:254 407:250 408:19 433:96 434:254 435:249 461:53 462:253 463:252 464:43 490:250 491:251 492:32 517:85 518:254 519:249 545:96 546:254 547:249 573:83 574:254 575:250 576:14 602:250 603:254 604:95 630:250 631:255 632:95 658:132 659:254 660:95 +1 124:32 125:253 126:31 152:32 153:251 154:149 180:32 181:251 182:188 208:32 209:251 210:188 236:32 237:251 238:228 239:59 264:32 265:253 266:253 267:95 292:28 293:236 294:251 295:114 321:127 322:251 323:251 349:127 350:251 351:251 377:48 378:232 379:251 406:223 407:253 408:159 434:221 435:251 436:158 462:142 463:251 464:158 490:64 491:251 492:242 493:55 518:64 519:251 520:253 521:161 546:64 547:253 548:255 549:221 574:16 575:181 576:253 577:220 603:79 604:253 605:236 606:63 632:213 633:251 634:126 660:96 661:251 662:126 +1 129:39 130:254 131:255 132:254 133:140 157:136 158:253 159:253 160:228 161:67 184:6 185:227 186:253 187:253 188:58 211:29 212:188 213:253 214:253 215:253 216:17 239:95 240:253 241:253 242:253 243:157 244:8 266:3 267:107 268:253 269:253 270:245 271:77 294:29 295:253 296:253 297:240 298:100 322:141 323:253 324:253 325:215 349:129 350:248 351:253 352:253 353:215 377:151 378:253 379:253 380:253 381:144 405:151 406:253 407:253 408:253 409:27 431:3 432:102 433:242 434:253 435:253 436:110 437:3 459:97 460:253 461:253 462:253 463:214 464:55 487:207 488:253 489:253 490:253 491:158 515:67 516:253 517:253 518:253 519:158 543:207 544:253 545:253 546:240 547:88 571:207 572:253 573:253 574:224 598:32 599:217 600:253 601:253 602:224 626:141 627:253 628:253 629:253 630:133 654:36 655:219 656:253 657:140 658:10 +0 123:59 124:55 149:71 150:192 151:254 152:250 153:147 154:17 176:123 177:247 178:253 179:254 180:253 181:253 182:196 183:79 184:176 185:175 186:175 187:124 188:48 203:87 204:247 205:247 206:176 207:95 208:102 209:117 210:243 211:237 212:192 213:232 214:253 215:253 216:245 217:152 218:6 230:23 231:229 232:253 233:138 238:219 239:58 241:95 242:118 243:80 244:230 245:254 246:196 247:30 258:120 259:254 260:205 261:8 266:114 272:38 273:255 274:254 275:155 276:5 286:156 287:253 288:92 301:61 302:235 303:253 304:102 314:224 315:253 316:78 330:117 331:253 332:196 333:18 342:254 343:253 344:78 358:9 359:211 360:253 361:73 370:254 371:253 372:78 387:175 388:253 389:155 398:194 399:254 400:101 415:79 416:254 417:155 426:112 427:253 428:211 429:9 443:73 444:251 445:200 454:41 455:241 456:253 457:87 471:25 472:240 473:253 483:147 484:253 485:227 486:47 499:94 500:253 501:200 511:5 512:193 513:253 514:230 515:76 527:175 528:253 529:155 540:31 541:219 542:254 543:255 544:126 545:18 553:14 554:149 555:254 556:244 557:45 569:21 570:158 571:254 572:253 573:226 574:162 575:118 576:96 577:20 578:20 579:73 580:118 581:224 582:253 583:247 584:85 598:30 599:155 600:253 601:253 602:253 603:253 604:254 605:253 606:253 607:253 608:253 609:254 610:247 611:84 627:5 628:27 629:117 630:206 631:244 632:229 633:213 634:213 635:213 636:176 637:117 638:32 659:45 660:23 +1 128:58 129:139 156:247 157:247 158:25 183:121 184:253 185:156 186:3 211:133 212:253 213:145 238:11 239:227 240:253 241:145 266:7 267:189 268:253 269:145 294:35 295:252 296:253 297:145 322:146 323:252 324:253 325:131 350:146 351:252 352:253 353:13 378:146 379:252 380:253 381:13 406:147 407:253 408:255 409:13 434:146 435:252 436:253 437:13 462:146 463:252 464:253 465:13 490:146 491:252 492:253 493:13 517:22 518:230 519:252 520:221 521:9 545:22 546:230 547:252 548:133 574:146 575:252 576:133 602:146 603:252 604:120 630:146 631:252 658:146 659:252 +1 129:28 130:247 131:255 132:165 156:47 157:221 158:252 159:252 160:164 184:177 185:252 186:252 187:252 188:164 212:177 213:252 214:252 215:223 216:78 240:177 241:252 242:252 243:197 267:114 268:236 269:252 270:235 271:42 294:5 295:148 296:252 297:252 298:230 321:14 322:135 323:252 324:252 325:252 326:230 349:78 350:252 351:252 352:252 353:252 354:162 377:78 378:252 379:252 380:252 381:252 382:9 405:78 406:252 407:252 408:252 409:252 410:9 432:32 433:200 434:252 435:252 436:252 437:105 438:3 459:10 460:218 461:252 462:252 463:252 464:105 465:8 487:225 488:252 489:252 490:252 491:240 492:69 514:44 515:237 516:252 517:252 518:228 519:85 541:59 542:218 543:252 544:252 545:225 546:93 568:65 569:208 570:252 571:252 572:252 573:175 596:133 597:252 598:252 599:252 600:225 601:68 624:133 625:252 626:252 627:244 628:54 652:133 653:252 654:252 655:48 +0 156:13 157:6 181:10 182:77 183:145 184:253 185:190 186:67 207:11 208:77 209:193 210:252 211:252 212:253 213:252 214:238 215:157 216:71 217:26 233:10 234:78 235:193 236:252 237:252 238:252 239:252 240:253 241:252 242:252 243:252 244:252 245:228 246:128 247:49 248:5 259:6 260:78 261:194 262:252 263:252 264:252 265:252 266:252 267:252 268:253 269:217 270:192 271:232 272:252 273:252 274:252 275:252 276:135 277:3 286:4 287:147 288:252 289:252 290:252 291:252 292:252 293:252 294:252 295:252 296:175 297:26 299:40 300:145 301:235 302:252 303:252 304:252 305:104 314:208 315:252 316:252 317:252 318:252 319:252 320:252 321:133 322:48 323:48 329:71 330:236 331:252 332:252 333:230 342:253 343:185 344:170 345:252 346:252 347:252 348:173 349:22 358:102 359:252 360:252 361:252 370:24 371:141 372:243 373:252 374:252 375:186 376:5 386:8 387:220 388:252 389:252 398:70 399:247 400:252 401:252 402:165 403:37 414:81 415:251 416:252 417:194 426:255 427:253 428:253 429:251 430:69 441:39 442:231 443:253 444:253 445:127 454:253 455:252 456:249 457:127 468:6 469:147 470:252 471:252 472:190 473:5 482:253 483:252 484:216 495:7 496:145 497:252 498:252 499:252 500:69 510:253 511:252 512:223 513:16 522:25 523:185 524:252 525:252 526:252 527:107 528:8 538:167 539:252 540:252 541:181 542:18 549:105 550:191 551:252 552:252 553:235 554:151 555:10 566:37 567:221 568:252 569:252 570:210 571:193 572:96 573:73 574:130 575:188 576:194 577:227 578:252 579:252 580:235 581:128 595:97 596:220 597:252 598:252 599:252 600:252 601:252 602:252 603:252 604:253 605:252 606:252 607:236 608:70 624:40 625:174 626:252 627:252 628:252 629:252 630:252 631:252 632:253 633:197 634:138 635:29 653:5 654:23 655:116 656:143 657:143 658:143 659:143 660:24 661:10 +0 127:28 128:164 129:254 130:233 131:148 132:11 154:3 155:164 156:254 157:234 158:225 159:254 160:204 182:91 183:254 184:235 185:48 186:32 187:166 188:251 189:92 208:33 209:111 210:214 211:205 212:49 215:24 216:216 217:210 235:34 236:217 237:254 238:254 239:211 244:87 245:237 246:43 262:34 263:216 264:254 265:254 266:252 267:243 268:61 272:38 273:248 274:182 290:171 291:254 292:184 293:205 294:175 295:36 301:171 302:227 317:28 318:234 319:190 320:13 321:193 322:157 329:124 330:238 331:26 345:140 346:254 347:131 349:129 350:157 357:124 358:254 359:95 373:201 374:238 375:56 377:70 378:103 385:124 386:254 387:148 400:62 401:255 402:210 413:150 414:254 415:122 428:86 429:254 430:201 431:15 440:28 441:237 442:246 443:44 456:128 457:254 458:143 468:34 469:243 470:227 484:62 485:254 486:210 496:58 497:249 498:179 512:30 513:240 514:210 524:207 525:254 526:64 541:216 542:231 543:34 551:129 552:248 553:170 554:9 569:131 570:254 571:170 577:17 578:129 579:248 580:225 581:24 597:50 598:245 599:245 600:184 601:106 602:106 603:106 604:133 605:231 606:254 607:244 608:53 626:67 627:249 628:254 629:254 630:254 631:254 632:254 633:251 634:193 635:40 655:38 656:157 657:248 658:166 659:166 660:139 661:57 +0 129:105 130:255 131:219 132:67 133:67 134:52 156:20 157:181 158:253 159:253 160:253 161:253 162:226 163:69 182:4 183:129 184:206 185:253 186:253 187:253 188:253 189:253 190:253 191:130 209:9 210:141 211:253 212:253 213:253 214:253 215:253 216:253 217:253 218:253 219:166 220:20 237:134 238:253 239:253 240:253 241:253 242:253 243:253 244:253 245:253 246:253 247:253 248:65 262:2 263:83 264:207 265:246 266:253 267:253 268:253 269:253 270:253 271:249 272:234 273:247 274:253 275:253 276:65 290:83 291:253 292:253 293:253 294:253 295:253 296:189 297:253 298:253 299:205 301:179 302:253 303:253 304:65 317:85 318:234 319:253 320:253 321:253 322:253 323:157 324:26 325:164 326:151 327:83 329:179 330:253 331:253 332:65 344:65 345:237 346:253 347:253 348:253 349:67 350:36 351:14 353:15 354:12 357:179 358:253 359:253 360:65 371:4 372:141 373:253 374:253 375:221 376:158 377:23 385:179 386:253 387:253 388:65 399:129 400:253 401:253 402:241 403:62 412:72 413:226 414:253 415:175 416:24 426:119 427:247 428:253 429:253 430:206 439:8 440:134 441:253 442:253 443:130 454:132 455:253 456:253 457:194 458:27 467:125 468:253 469:253 470:253 471:130 481:45 482:213 483:253 484:253 485:112 493:70 494:170 495:247 496:253 497:253 498:89 499:43 509:67 510:253 511:253 512:196 513:55 514:9 520:8 521:131 522:253 523:253 524:253 525:86 526:1 537:67 538:253 539:253 540:253 541:253 542:129 546:43 547:114 548:134 549:253 550:253 551:231 552:139 553:41 565:20 566:167 567:253 568:253 569:253 570:247 571:179 572:179 573:179 574:206 575:253 576:253 577:253 578:253 579:72 594:103 595:240 596:253 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:244 605:119 606:8 607:1 623:107 624:253 625:253 626:253 627:253 628:253 629:253 630:253 631:175 632:111 651:3 652:121 653:253 654:253 655:253 656:253 657:253 658:182 659:24 +0 125:22 126:183 127:252 128:254 129:252 130:252 131:252 132:76 151:85 152:85 153:168 154:250 155:250 156:252 157:250 158:250 159:250 160:250 161:71 163:43 164:85 165:14 178:107 179:252 180:250 181:250 182:250 183:250 184:252 185:250 186:250 187:250 188:250 189:210 191:127 192:250 193:146 205:114 206:237 207:252 208:250 209:250 210:250 211:250 212:252 213:250 214:250 215:250 216:250 217:210 219:127 220:250 221:250 232:107 233:237 234:250 235:252 236:250 237:250 238:250 239:74 240:41 241:41 242:41 243:41 244:217 245:34 247:127 248:250 249:250 259:15 260:148 261:252 262:252 263:254 264:238 265:105 275:128 276:252 277:252 286:15 287:140 288:250 289:250 290:250 291:167 292:111 303:127 304:250 305:250 314:43 315:250 316:250 317:250 318:250 331:127 332:250 333:250 342:183 343:250 344:250 345:250 346:110 358:57 359:210 360:250 361:250 370:252 371:250 372:250 373:110 374:7 386:85 387:250 388:250 389:250 398:254 399:252 400:252 401:83 414:86 415:252 416:252 417:217 426:252 427:250 428:250 429:138 430:14 441:15 442:140 443:250 444:250 445:41 454:252 455:250 456:250 457:250 458:41 469:43 470:250 471:250 472:250 473:41 482:252 483:250 484:250 485:250 486:181 497:183 498:250 499:250 500:250 501:41 510:76 511:250 512:250 513:250 514:250 524:177 525:252 526:250 527:250 528:110 529:7 538:36 539:224 540:252 541:252 542:252 543:219 544:43 545:43 546:43 547:7 549:15 550:43 551:183 552:252 553:255 554:252 555:126 567:85 568:250 569:250 570:250 571:252 572:250 573:250 574:250 575:111 576:86 577:140 578:250 579:250 580:250 581:252 582:222 583:83 595:42 596:188 597:250 598:250 599:252 600:250 601:250 602:250 603:250 604:252 605:250 606:250 607:250 608:250 609:126 610:83 624:127 625:250 626:250 627:252 628:250 629:250 630:250 631:250 632:252 633:250 634:250 635:137 636:83 652:21 653:41 654:217 655:252 656:250 657:250 658:250 659:250 660:217 661:41 662:41 663:14 +1 155:114 156:206 157:25 183:238 184:252 185:55 211:222 212:252 213:55 239:113 240:252 241:55 267:113 268:252 269:55 295:255 296:253 297:56 323:253 324:176 325:6 350:32 351:253 352:233 353:43 378:140 379:253 380:195 381:19 406:140 407:253 408:167 433:29 434:253 435:141 461:29 462:252 463:140 489:29 490:252 491:140 517:29 518:252 519:140 545:29 546:252 547:140 573:169 574:253 575:79 601:169 602:252 628:76 629:234 630:141 656:197 657:233 658:37 684:197 685:223 +1 127:73 128:253 129:253 130:63 155:115 156:252 157:252 158:144 183:217 184:252 185:252 186:144 210:63 211:237 212:252 213:252 214:144 238:109 239:252 240:252 241:252 266:109 267:252 268:252 269:252 294:109 295:252 296:252 297:252 322:191 323:252 324:252 325:252 349:145 350:255 351:253 352:253 353:253 376:32 377:237 378:253 379:252 380:252 381:210 404:37 405:252 406:253 407:252 408:252 409:108 432:37 433:252 434:253 435:252 436:252 437:108 460:21 461:207 462:255 463:253 464:253 465:108 489:144 490:253 491:252 492:252 493:108 516:27 517:221 518:253 519:252 520:252 521:108 544:16 545:190 546:253 547:252 548:252 549:108 573:145 574:255 575:253 576:253 577:253 601:144 602:253 603:252 604:252 605:210 629:144 630:253 631:252 632:252 633:108 657:62 658:253 659:252 660:252 661:108 +1 120:85 121:253 122:132 123:9 147:82 148:241 149:251 150:251 151:128 175:175 176:251 177:251 178:251 179:245 180:121 203:13 204:204 205:251 206:251 207:251 208:245 209:107 232:39 233:251 234:251 235:251 236:251 237:167 238:22 260:15 261:155 262:251 263:251 264:251 265:251 266:177 289:15 290:157 291:248 292:251 293:251 294:251 295:165 319:214 320:251 321:251 322:251 323:212 324:78 325:24 347:109 348:251 349:251 350:251 351:253 352:251 353:170 354:10 375:5 376:57 377:162 378:251 379:253 380:251 381:251 382:18 405:106 406:239 407:255 408:253 409:253 410:213 434:105 435:253 436:251 437:251 438:230 439:72 463:253 464:251 465:251 466:251 467:221 468:67 491:72 492:251 493:251 494:251 495:251 496:96 519:36 520:199 521:251 522:251 523:251 524:155 525:15 548:45 549:204 550:251 551:251 552:251 553:157 577:161 578:249 579:251 580:251 581:248 582:147 606:233 607:251 608:251 609:251 610:173 634:233 635:251 636:251 637:251 638:173 662:53 663:131 664:251 665:251 666:173 +1 126:15 127:200 128:255 129:90 154:42 155:254 156:254 157:173 182:42 183:254 184:254 185:199 210:26 211:237 212:254 213:221 214:12 239:213 240:254 241:231 242:17 267:213 268:254 269:199 295:213 296:254 297:199 323:213 324:254 325:96 350:20 351:232 352:254 353:33 378:84 379:254 380:229 381:17 406:168 407:254 408:203 433:8 434:217 435:254 436:187 461:84 462:254 463:254 464:48 489:195 490:254 491:254 492:37 516:20 517:233 518:254 519:212 520:4 544:132 545:254 546:254 547:82 571:9 572:215 573:254 574:254 575:116 576:46 599:55 600:254 601:254 602:254 603:254 604:121 627:113 628:254 629:254 630:254 631:254 632:40 655:12 656:163 657:254 658:185 659:58 660:1 +0 182:32 183:57 184:57 185:57 186:57 187:57 188:57 189:57 208:67 209:185 210:229 211:252 212:252 213:252 214:253 215:252 216:252 217:252 218:185 219:66 234:13 235:188 236:246 237:252 238:253 239:252 240:252 241:252 242:241 243:139 244:177 245:252 246:253 247:246 248:187 249:13 261:26 262:255 263:253 264:244 265:175 266:101 274:126 275:244 276:253 277:153 288:82 289:243 290:253 291:214 292:81 303:169 304:252 305:252 315:19 316:215 317:252 318:206 319:56 331:169 332:252 333:252 343:157 344:252 345:252 346:13 359:169 360:252 361:151 370:41 371:253 372:253 373:128 386:92 387:253 388:206 389:13 398:166 399:252 400:196 401:9 414:216 415:252 416:142 426:253 427:252 428:168 441:89 442:253 443:208 444:13 454:253 455:252 456:68 468:38 469:225 470:253 471:96 482:254 483:253 484:56 495:45 496:229 497:253 498:151 510:253 511:252 512:81 522:70 523:225 524:252 525:227 538:216 539:252 540:168 548:29 549:134 550:253 551:252 552:186 553:31 566:91 567:252 568:243 569:125 573:51 574:114 575:113 576:210 577:252 578:253 579:151 580:19 595:157 596:253 597:253 598:254 599:253 600:253 601:253 602:254 603:253 604:244 605:175 606:51 623:19 624:122 625:196 626:197 627:221 628:196 629:196 630:197 631:121 632:56 655:25 +0 127:42 128:235 129:255 130:84 153:15 154:132 155:208 156:253 157:253 158:171 159:108 180:6 181:177 182:253 183:253 184:253 185:253 186:253 187:242 188:110 208:151 209:253 210:253 211:253 212:253 213:253 214:253 215:253 216:139 235:48 236:208 237:253 238:253 239:253 240:253 241:253 242:253 243:253 244:139 263:85 264:253 265:253 266:253 267:253 268:236 269:156 270:184 271:253 272:148 273:6 290:7 291:141 292:253 293:253 294:253 295:253 296:27 298:170 299:253 300:253 301:74 318:19 319:253 320:253 321:253 322:253 323:253 324:27 326:170 327:253 328:253 329:74 345:16 346:186 347:253 348:253 349:253 350:242 351:105 352:4 354:170 355:253 356:253 357:94 358:1 373:141 374:253 375:253 376:253 377:242 378:100 382:170 383:253 384:253 385:253 386:8 401:141 402:253 403:253 404:253 405:224 410:170 411:253 412:253 413:253 414:8 428:12 429:158 430:253 431:253 432:230 433:51 438:18 439:237 440:253 441:253 442:8 456:76 457:253 458:253 459:218 460:61 467:236 468:253 469:253 470:8 484:76 485:253 486:253 487:168 495:110 496:253 497:132 498:3 512:76 513:253 514:253 515:168 521:20 522:174 523:239 524:147 525:5 539:5 540:155 541:253 542:253 543:168 548:102 549:170 550:253 551:253 552:139 567:3 568:128 569:253 570:253 571:228 572:179 573:179 574:179 575:179 576:245 577:253 578:253 579:219 580:41 596:76 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:253 605:253 606:253 607:163 624:39 625:199 626:253 627:253 628:253 629:253 630:253 631:253 632:253 633:253 634:170 635:9 653:36 654:219 655:253 656:253 657:253 658:253 659:253 660:224 661:65 662:22 +1 156:202 157:253 158:69 184:253 185:252 186:121 212:253 213:252 214:69 240:253 241:252 242:69 267:106 268:253 269:231 270:37 295:179 296:255 297:196 322:17 323:234 324:253 325:92 350:93 351:252 352:253 353:92 378:93 379:252 380:253 381:92 406:93 407:252 408:232 409:8 434:208 435:253 436:116 462:207 463:252 464:116 490:207 491:252 492:32 517:57 518:244 519:252 545:122 546:252 547:252 573:185 574:253 575:253 601:184 602:252 603:252 629:101 630:252 631:252 657:13 658:173 659:252 660:43 686:9 687:232 688:116 +1 156:73 157:253 158:253 159:253 160:124 184:73 185:251 186:251 187:251 188:251 212:99 213:251 214:251 215:251 216:225 240:253 241:251 242:251 243:251 244:71 266:79 267:180 268:253 269:251 270:251 271:173 272:20 294:110 295:253 296:255 297:253 298:216 322:109 323:251 324:253 325:251 326:215 350:109 351:251 352:253 353:251 354:215 378:109 379:251 380:253 381:251 382:137 406:109 407:251 408:253 409:251 410:35 433:37 434:253 435:253 436:255 437:253 438:35 461:140 462:251 463:251 464:253 465:168 466:15 488:125 489:246 490:251 491:251 492:190 493:15 516:144 517:251 518:251 519:251 520:180 543:53 544:221 545:251 546:251 547:251 548:51 571:125 572:253 573:253 574:253 575:201 598:105 599:253 600:251 601:251 602:188 603:30 626:180 627:253 628:251 629:251 630:142 654:180 655:253 656:251 657:235 658:82 682:180 683:253 684:251 685:215 +1 124:111 125:255 126:48 152:162 153:253 154:237 155:63 180:206 181:253 182:253 183:183 208:87 209:217 210:253 211:205 237:90 238:253 239:238 240:60 265:37 266:225 267:253 268:89 294:206 295:253 296:159 322:206 323:253 324:226 350:206 351:253 352:226 378:206 379:253 380:226 406:206 407:253 408:226 434:206 435:253 436:226 462:206 463:253 464:226 490:206 491:253 492:226 518:206 519:253 520:237 521:45 546:206 547:253 548:253 549:109 574:173 575:253 576:253 577:109 602:69 603:253 604:253 605:109 630:64 631:248 632:253 633:109 659:112 660:253 661:109 +0 99:70 100:255 101:165 102:114 127:122 128:253 129:253 130:253 131:120 155:165 156:253 157:253 158:253 159:234 160:52 183:99 184:253 185:253 186:253 187:253 188:228 189:26 209:60 210:168 211:238 212:202 213:174 214:253 215:253 216:253 217:127 235:91 236:81 237:1 238:215 239:128 240:28 241:12 242:181 243:253 244:253 245:175 246:3 262:18 263:204 264:253 265:77 270:7 271:253 272:253 273:253 274:54 289:54 290:248 291:253 292:253 293:143 298:1 299:127 300:253 301:253 302:188 317:104 318:253 319:253 320:253 321:20 327:81 328:249 329:253 330:191 345:192 346:253 347:253 348:218 349:5 356:203 357:253 358:208 359:21 372:56 373:237 374:253 375:250 376:100 384:104 385:253 386:253 387:75 400:76 401:253 402:253 403:224 412:119 413:253 414:253 415:75 428:80 429:253 430:253 431:103 439:4 440:241 441:253 442:218 443:32 456:213 457:253 458:253 459:103 467:125 468:253 469:253 470:191 484:213 485:253 486:253 487:103 494:3 495:176 496:253 497:253 498:135 512:213 513:253 514:253 515:103 521:9 522:162 523:253 524:253 525:226 526:37 540:179 541:253 542:253 543:135 548:46 549:157 550:253 551:253 552:253 553:63 568:23 569:188 570:253 571:249 572:179 573:179 574:179 575:179 576:233 577:253 578:253 579:233 580:156 581:10 597:51 598:235 599:253 600:253 601:253 602:253 603:253 604:253 605:251 606:232 607:120 626:16 627:124 628:253 629:253 630:253 631:253 632:152 633:104 +1 124:29 125:197 126:255 127:84 152:85 153:251 154:253 155:83 180:86 181:253 182:254 183:253 208:85 209:251 210:253 211:251 236:86 237:253 238:254 239:253 240:169 264:85 265:251 266:253 267:251 268:168 292:86 293:253 294:254 295:253 296:169 320:28 321:196 322:253 323:251 324:168 349:169 350:254 351:253 352:169 377:168 378:253 379:251 380:168 405:169 406:254 407:253 408:169 433:168 434:253 435:251 436:168 462:254 463:253 464:254 465:139 490:253 491:251 492:253 493:251 518:254 519:253 520:254 521:253 522:57 546:253 547:251 548:253 549:251 550:168 574:198 575:253 576:254 577:253 578:114 602:85 603:251 604:253 605:251 630:85 631:253 632:254 633:253 658:28 659:83 660:196 661:83 +1 159:31 160:210 161:253 162:163 187:198 188:252 189:252 190:162 213:10 214:86 215:242 216:252 217:252 218:66 241:164 242:252 243:252 244:252 245:188 246:8 268:53 269:242 270:252 271:252 272:225 273:14 296:78 297:252 298:252 299:252 300:204 323:56 324:231 325:252 326:252 327:212 328:35 351:157 352:252 353:252 354:252 355:37 377:8 378:132 379:253 380:252 381:252 382:230 383:24 405:45 406:252 407:253 408:252 409:154 410:55 427:7 428:55 433:107 434:253 435:255 436:228 437:53 454:15 455:24 456:23 460:110 461:242 462:252 463:228 464:59 482:57 483:83 487:88 488:247 489:252 490:252 491:140 514:15 515:189 516:252 517:252 518:252 542:74 543:252 544:252 545:238 546:90 570:178 571:252 572:252 573:189 597:40 598:217 599:252 600:252 601:59 625:75 626:252 627:252 628:252 629:85 630:61 653:62 654:239 655:252 656:156 657:14 682:178 683:252 684:14 +1 131:159 132:255 133:122 158:167 159:228 160:253 161:121 185:64 186:236 187:251 188:205 189:110 212:48 213:158 214:251 215:251 216:178 217:39 240:190 241:251 242:251 243:251 267:96 268:253 269:253 270:253 271:153 295:194 296:251 297:251 298:211 299:74 322:80 323:174 324:251 325:251 326:140 327:47 349:16 350:181 351:253 352:251 353:219 354:23 377:64 378:251 379:253 380:251 381:204 382:19 405:223 406:253 407:255 408:233 409:48 431:20 432:174 433:244 434:251 435:253 436:109 437:31 459:96 460:189 461:251 462:251 463:126 464:31 486:24 487:106 488:251 489:235 490:188 491:100 514:96 515:251 516:251 517:228 518:59 542:255 543:253 544:253 545:213 546:36 569:100 570:253 571:251 572:251 573:85 574:23 596:32 597:127 598:253 599:235 600:126 601:15 624:104 625:251 626:253 627:240 628:79 652:83 653:193 654:253 655:220 +0 153:92 154:191 155:178 156:253 157:242 158:141 159:104 160:29 180:26 181:253 182:252 183:252 184:252 185:253 186:252 187:252 188:252 189:108 190:19 206:57 207:123 208:222 209:253 210:252 211:252 212:252 213:168 214:224 215:252 216:252 217:253 218:84 233:176 234:243 235:252 236:252 237:253 238:252 239:252 240:252 242:19 243:153 244:252 245:253 246:209 247:25 259:10 260:128 261:255 262:253 263:244 264:225 265:114 266:194 267:253 268:178 272:163 273:254 274:253 275:168 287:85 288:252 289:253 290:189 291:56 294:19 295:133 296:9 300:38 301:253 302:252 303:168 314:19 315:191 316:252 317:194 318:19 329:253 330:252 331:234 332:22 342:107 343:252 344:252 345:13 357:253 358:252 359:252 360:128 370:169 371:253 372:241 385:141 386:253 387:253 388:140 397:19 398:225 399:252 400:139 413:66 414:252 415:252 416:139 425:29 426:252 427:252 428:52 441:29 442:252 443:252 444:139 453:29 454:252 455:252 456:28 469:29 470:252 471:252 472:40 481:141 482:253 483:253 484:91 497:154 498:253 499:168 509:66 510:252 511:252 512:165 525:253 526:252 527:168 537:19 538:224 539:252 540:252 552:126 553:253 554:252 555:80 566:169 567:252 568:252 569:214 570:38 579:126 580:249 581:253 582:151 583:6 594:26 595:223 596:253 597:254 598:253 599:128 600:29 604:13 605:41 606:216 607:253 608:253 609:226 610:38 623:122 624:252 625:253 626:252 627:252 628:252 629:169 630:169 631:169 632:206 633:253 634:252 635:252 636:202 637:38 651:19 652:56 653:168 654:224 655:252 656:252 657:253 658:252 659:252 660:252 661:253 662:233 663:130 664:6 682:94 683:139 684:190 685:153 686:252 687:164 688:139 689:28 690:22 +1 128:53 129:250 130:255 131:25 156:167 157:253 158:253 159:25 182:3 183:123 184:247 185:253 186:253 187:25 210:9 211:253 212:253 213:253 214:253 215:25 238:9 239:253 240:253 241:253 242:253 243:25 266:9 267:253 268:253 269:253 270:180 271:13 294:9 295:253 296:253 297:253 298:104 322:9 323:253 324:253 325:253 326:104 350:15 351:253 352:253 353:253 354:104 378:184 379:253 380:253 381:228 382:68 406:184 407:253 408:253 409:182 433:103 434:251 435:253 436:253 437:12 461:106 462:253 463:253 464:253 465:8 488:24 489:238 490:253 491:253 492:253 493:8 516:27 517:253 518:253 519:253 520:253 521:8 544:27 545:253 546:253 547:253 548:253 549:8 572:27 573:253 574:253 575:253 576:177 577:4 600:160 601:253 602:253 603:253 604:87 628:202 629:253 630:253 631:219 632:54 656:81 657:253 658:247 659:51 +0 122:63 123:176 124:253 125:253 126:159 127:113 128:63 150:140 151:253 152:252 153:252 154:252 155:252 156:241 157:100 158:66 177:54 178:227 179:253 180:252 181:252 182:252 183:252 184:253 185:252 186:239 187:181 188:57 204:38 205:224 206:252 207:253 208:226 209:246 210:252 211:252 212:253 213:252 214:252 215:252 216:252 217:108 218:3 232:57 233:252 234:252 235:253 236:27 237:88 238:112 239:112 240:112 241:112 242:142 243:252 244:252 245:253 246:152 247:31 260:198 261:253 262:253 263:79 270:32 271:153 272:253 273:255 274:253 275:196 287:76 288:246 289:252 290:127 299:3 300:106 301:253 302:252 303:214 304:28 315:194 316:252 317:252 318:112 329:143 330:252 331:252 332:193 343:225 344:252 345:217 346:37 357:38 358:234 359:252 360:223 370:63 371:240 372:252 373:84 386:146 387:252 388:223 398:114 399:253 400:228 401:47 414:147 415:253 416:253 417:112 426:159 427:252 428:195 442:225 443:252 444:252 445:112 454:253 455:252 456:195 470:225 471:252 472:230 473:25 482:159 483:252 484:202 485:10 497:92 498:243 499:252 500:208 510:113 511:252 512:252 513:161 524:79 525:253 526:252 527:220 528:37 538:114 539:253 540:253 541:253 542:174 543:63 550:26 551:128 552:253 553:255 554:253 555:133 566:12 567:228 568:252 569:252 570:252 571:241 572:100 573:85 574:76 576:85 577:131 578:231 579:252 580:252 581:253 582:129 583:6 595:97 596:208 597:252 598:252 599:253 600:252 601:252 602:246 603:197 604:253 605:252 606:252 607:252 608:220 609:133 610:6 624:19 625:99 626:239 627:253 628:252 629:252 630:252 631:252 632:253 633:252 634:245 635:223 636:99 654:63 655:112 656:112 657:221 658:252 659:252 660:253 661:127 662:87 +0 153:12 154:136 155:254 156:255 157:195 158:115 159:3 180:6 181:175 182:253 183:196 184:160 185:252 186:253 187:15 208:130 209:253 210:234 211:4 213:27 214:205 215:232 216:40 235:54 236:246 237:253 238:68 242:24 243:243 244:106 262:3 263:134 264:235 265:99 266:4 271:132 272:247 273:77 290:56 291:253 292:62 299:23 300:233 301:129 318:179 319:183 320:4 328:182 329:220 345:21 346:232 347:59 356:95 357:232 358:21 373:128 374:183 385:228 386:85 401:187 402:124 413:228 414:186 429:187 430:124 441:228 442:104 457:187 458:124 469:169 470:184 485:187 486:124 497:203 498:150 513:187 514:124 524:10 525:220 526:39 541:187 542:155 552:111 553:201 569:129 570:228 571:7 579:12 580:181 581:76 598:234 599:166 600:9 606:24 607:209 608:106 626:139 627:250 628:167 629:11 630:2 631:11 632:11 633:129 634:227 635:90 636:11 655:95 656:247 657:253 658:178 659:253 660:253 661:244 662:86 684:47 685:175 686:253 687:232 688:149 689:40 +1 128:255 129:253 130:57 156:253 157:251 158:225 159:56 183:169 184:254 185:253 186:254 187:84 211:168 212:253 213:251 214:253 215:83 238:85 239:253 240:254 241:253 242:169 266:85 267:251 268:253 269:251 270:56 294:141 295:253 296:254 297:253 322:253 323:251 324:253 325:251 350:254 351:253 352:254 353:253 378:253 379:251 380:253 381:251 406:254 407:253 408:254 409:196 433:114 434:253 435:251 436:253 437:83 461:169 462:254 463:253 464:226 465:56 489:168 490:253 491:251 492:168 516:85 517:253 518:254 519:253 544:85 545:251 546:253 547:251 572:254 573:253 574:254 575:253 600:253 601:251 602:253 603:251 628:254 629:253 630:254 631:253 656:139 657:251 658:253 659:138 +0 151:23 152:167 153:208 154:254 155:255 156:129 157:19 179:151 180:253 181:253 182:253 183:253 184:253 185:209 186:26 207:181 208:253 209:253 210:253 211:227 212:181 213:253 214:207 215:22 235:227 236:253 237:253 238:253 239:92 240:38 241:226 242:253 243:129 244:2 263:193 264:253 265:253 266:248 267:62 269:50 270:253 271:253 272:45 291:170 292:253 293:253 294:135 297:12 298:208 299:253 300:119 318:16 319:232 320:253 321:253 322:21 326:60 327:253 328:185 346:164 347:253 348:253 349:224 350:14 354:14 355:217 356:247 357:62 373:3 374:193 375:253 376:250 377:64 383:199 384:253 385:179 401:67 402:253 403:253 404:205 411:98 412:253 413:188 429:151 430:253 431:245 432:43 439:63 440:250 441:188 457:151 458:253 459:243 468:244 469:222 470:22 485:151 486:253 487:217 496:244 497:253 498:115 512:3 513:195 514:253 515:134 524:156 525:253 526:150 541:140 542:253 543:134 552:239 553:253 554:139 569:44 570:253 571:134 579:53 580:246 581:237 582:32 597:8 598:200 599:229 600:40 606:25 607:225 608:253 609:188 626:120 627:250 628:230 629:58 630:17 632:12 633:42 634:213 635:253 636:238 637:84 655:151 656:253 657:253 658:217 659:179 660:206 661:253 662:253 663:196 664:118 683:18 684:58 685:145 686:152 687:253 688:214 689:145 690:74 691:7 +1 130:24 131:150 132:233 133:38 156:14 157:89 158:253 159:254 160:254 161:71 183:78 184:203 185:254 186:254 187:254 188:232 189:77 190:54 191:8 209:12 210:155 211:240 212:254 213:223 214:76 215:254 216:254 217:254 218:254 219:68 235:3 236:101 237:216 238:254 239:227 240:122 241:26 242:110 243:254 244:254 245:254 246:184 247:100 262:46 263:222 264:254 265:254 266:179 267:48 270:181 271:254 272:254 273:146 274:6 288:2 289:145 290:248 291:254 292:182 293:111 294:4 297:3 298:250 299:254 300:206 301:3 315:6 316:144 317:254 318:254 319:171 325:125 326:254 327:252 328:80 342:6 343:142 344:254 345:179 346:95 347:4 352:61 353:246 354:254 355:150 370:64 371:254 372:177 373:14 380:124 381:254 382:246 383:32 398:108 399:97 400:15 407:24 408:226 409:254 410:116 435:177 436:255 437:254 438:5 463:196 464:254 465:99 466:1 490:3 491:199 492:254 493:79 518:129 519:254 520:254 521:23 546:178 547:254 548:192 549:8 550:3 551:43 573:11 574:198 575:254 576:128 577:66 578:130 579:225 595:137 596:202 597:106 598:84 599:84 600:84 601:112 602:254 603:254 604:254 605:254 606:212 607:151 623:172 624:254 625:254 626:254 627:254 628:254 629:254 630:254 631:254 632:254 633:162 634:75 651:12 652:106 653:177 654:254 655:254 656:254 657:235 658:135 659:100 660:17 661:2 +0 125:120 126:253 127:253 128:63 151:38 152:131 153:246 154:252 155:252 156:203 157:15 179:222 180:252 181:252 182:252 183:252 184:166 185:38 205:4 206:107 207:253 208:252 209:252 210:252 211:252 212:253 213:224 214:137 215:26 233:107 234:252 235:253 236:252 237:220 238:128 239:252 240:253 241:252 242:252 243:239 244:140 261:170 262:253 263:255 264:168 267:79 268:192 269:253 270:253 271:253 272:253 273:255 274:90 288:51 289:243 290:252 291:215 292:33 296:12 297:74 298:233 299:252 300:252 301:253 302:195 303:19 316:166 317:252 318:252 319:31 326:43 327:149 328:195 329:253 330:252 331:177 332:19 343:57 344:234 345:252 346:252 357:237 358:252 359:252 360:180 361:13 371:85 372:252 373:252 374:173 385:50 386:237 387:252 388:252 389:112 399:226 400:253 401:240 402:63 414:163 415:253 416:253 417:112 426:38 427:234 428:252 429:176 442:85 443:252 444:252 445:158 454:113 455:252 456:252 457:84 470:19 471:209 472:252 473:252 482:207 483:252 484:252 485:84 498:10 499:203 500:252 501:236 510:253 511:252 512:252 513:84 526:85 527:252 528:252 529:112 538:114 539:253 540:253 541:146 553:51 554:159 555:253 556:240 557:63 566:75 567:243 568:252 569:249 570:146 579:57 580:85 581:238 582:252 583:252 584:99 595:116 596:252 597:252 598:252 599:198 600:197 601:165 602:57 603:57 604:57 605:182 606:197 607:234 608:252 609:253 610:233 611:164 612:19 623:28 624:84 625:180 626:252 627:253 628:252 629:252 630:252 631:252 632:253 633:252 634:252 635:252 636:252 637:225 638:71 653:13 654:112 655:253 656:252 657:252 658:252 659:252 660:253 661:252 662:252 663:157 664:112 +1 127:155 128:253 129:126 155:253 156:251 157:141 158:4 183:253 184:251 185:251 186:31 211:253 212:251 213:251 214:31 239:253 240:251 241:251 242:31 267:255 268:253 269:253 270:31 293:8 294:131 295:253 296:251 297:235 298:27 321:64 322:251 323:253 324:251 325:126 349:64 350:251 351:253 352:251 353:126 377:64 378:251 379:253 380:251 381:126 405:64 406:253 407:255 408:221 433:182 434:251 435:253 436:200 460:64 461:236 462:251 463:253 464:62 487:8 488:158 489:251 490:251 491:169 492:8 515:32 516:251 517:251 518:251 519:158 543:32 544:253 545:253 546:253 547:159 571:32 572:251 573:251 574:251 575:39 599:32 600:251 601:251 602:251 627:32 628:251 629:251 630:251 631:100 655:32 656:251 657:251 658:251 +0 101:88 102:127 103:5 126:19 127:58 128:20 129:14 130:217 131:19 152:7 153:146 154:247 155:253 156:235 157:27 158:84 159:81 180:126 181:253 182:164 183:19 184:15 187:156 188:9 208:214 209:222 210:34 215:234 216:58 235:59 236:254 237:116 243:235 244:58 263:141 264:251 265:72 271:151 272:140 291:224 292:233 299:136 300:223 319:254 320:218 327:136 328:253 347:254 348:135 355:136 356:253 374:23 375:255 376:114 383:137 384:231 402:98 403:254 404:122 411:136 412:155 430:98 431:254 432:106 439:166 440:155 458:98 459:254 460:128 467:234 468:193 486:98 487:254 488:135 494:61 495:248 496:118 515:255 516:238 517:18 521:13 522:224 523:254 524:58 543:201 544:253 545:128 546:2 548:5 549:150 550:253 551:167 552:9 571:18 572:226 573:253 574:49 575:31 576:156 577:253 578:228 579:13 600:147 601:253 602:243 603:241 604:254 605:227 606:43 628:5 629:126 630:245 631:253 632:231 633:46 +0 127:37 128:141 129:156 130:156 131:194 132:194 133:47 153:11 154:132 155:239 156:253 157:253 158:253 159:253 160:254 161:181 180:25 181:172 182:253 183:235 184:167 185:78 186:93 187:174 188:254 189:247 190:54 207:26 208:210 209:253 210:237 211:90 216:201 217:253 218:78 235:192 236:253 237:237 238:58 244:156 245:253 246:78 262:141 263:254 264:235 265:53 269:19 270:5 272:156 273:254 274:78 289:46 290:254 291:253 292:92 296:17 297:226 298:217 299:49 300:148 301:253 302:78 317:165 318:254 319:239 320:24 324:20 325:253 326:253 327:58 328:18 329:115 330:24 344:37 345:248 346:254 347:91 352:2 353:117 354:250 355:163 356:91 372:77 373:253 374:254 375:39 382:196 383:253 384:173 400:159 401:254 402:218 403:15 410:77 411:254 412:255 413:61 428:234 429:253 430:113 438:21 439:226 440:254 441:135 455:25 456:240 457:253 458:68 467:195 468:254 469:135 483:79 484:253 485:253 495:195 496:254 497:135 511:79 512:253 513:253 514:76 523:195 524:254 525:99 540:212 541:254 542:209 543:9 550:10 551:209 552:196 553:15 568:54 569:253 570:254 571:137 572:36 576:2 577:20 578:168 579:253 580:60 596:28 597:235 598:254 599:253 600:199 601:124 602:79 603:79 604:167 605:253 606:253 607:185 608:30 625:15 626:117 627:217 628:253 629:253 630:253 631:254 632:253 633:240 634:109 635:12 655:27 656:126 657:208 658:253 659:193 660:147 661:40 +0 154:32 155:134 156:218 157:254 158:254 159:254 160:217 161:84 176:44 177:208 178:215 179:156 180:35 181:119 182:236 183:246 184:136 185:91 186:69 187:151 188:249 189:246 190:78 203:44 204:230 205:254 206:254 207:254 208:254 209:254 210:196 211:48 216:60 217:224 218:210 219:24 231:118 232:254 233:202 234:19 235:201 236:254 237:181 238:9 245:35 246:233 247:168 259:193 260:223 261:34 263:59 264:163 265:236 266:15 274:140 275:205 276:8 286:60 287:254 288:176 293:38 302:54 303:237 304:80 314:59 315:254 316:93 331:131 332:200 342:59 343:240 344:24 359:79 360:214 370:59 371:234 387:67 388:248 389:54 398:59 399:234 416:235 417:58 426:60 427:235 443:79 444:255 445:59 454:59 455:251 456:66 471:79 472:250 473:54 482:59 483:254 484:108 499:146 500:214 510:5 511:203 512:187 513:3 526:4 527:188 528:199 539:118 540:254 541:57 554:96 555:254 556:117 567:16 568:237 569:224 570:14 581:14 582:187 583:206 584:8 596:88 597:252 598:186 599:16 608:16 609:187 610:252 611:125 625:100 626:254 627:237 628:94 629:24 635:13 636:214 637:254 638:166 653:3 654:57 655:215 656:248 657:241 658:235 659:197 660:137 661:137 662:137 663:231 664:238 665:155 666:25 684:57 685:155 686:246 687:254 688:254 689:254 690:254 691:147 692:36 +1 124:102 125:252 126:252 127:41 152:102 153:250 154:250 155:202 180:102 181:250 182:250 183:232 184:91 208:102 209:250 210:250 211:212 212:29 236:102 237:252 238:252 239:254 240:150 264:102 265:250 266:250 267:252 268:149 292:102 293:250 294:250 295:252 296:149 320:102 321:250 322:250 323:252 324:231 325:80 349:152 350:252 351:254 352:252 353:100 377:151 378:250 379:252 380:250 381:100 405:151 406:250 407:252 408:250 409:100 433:151 434:250 435:252 436:250 437:100 461:123 462:243 463:254 464:252 465:100 490:202 491:252 492:250 493:100 518:80 519:252 520:250 521:190 522:30 547:252 548:250 549:250 550:49 575:255 576:252 577:252 578:252 579:214 580:31 603:171 604:250 605:250 606:250 607:252 608:190 609:40 631:20 632:160 633:250 634:250 635:252 636:250 637:100 660:20 661:170 662:250 663:212 664:49 665:20 +0 124:20 125:121 126:197 127:253 128:64 151:23 152:200 153:252 154:252 155:252 156:184 157:6 178:25 179:197 180:252 181:252 182:252 183:252 184:253 185:228 186:107 187:15 205:26 206:196 207:252 208:252 209:252 210:252 211:252 212:253 213:252 214:252 215:219 216:178 217:21 233:186 234:252 235:238 236:94 237:67 238:224 239:217 240:53 241:109 242:245 243:252 244:252 245:213 246:63 260:98 261:242 262:252 263:101 266:39 267:31 270:109 271:128 272:241 273:252 274:207 275:97 287:17 288:230 289:252 290:241 291:56 300:109 301:252 302:252 303:229 304:17 314:13 315:192 316:252 317:243 318:96 328:25 329:127 330:252 331:252 332:120 342:121 343:252 344:252 345:165 357:125 358:252 359:252 360:246 361:70 370:190 371:252 372:252 373:39 385:26 386:210 387:252 388:252 389:119 398:255 399:253 400:159 414:22 415:209 416:253 417:183 426:253 427:252 428:103 443:34 444:252 445:252 454:253 455:252 456:26 471:27 472:252 473:252 482:253 483:252 484:168 485:13 499:70 500:252 501:209 510:147 511:252 512:252 513:75 526:68 527:233 528:252 529:119 538:121 539:252 540:252 541:189 542:40 552:15 553:82 554:231 555:252 556:214 557:31 566:38 567:135 568:248 569:252 570:231 571:145 572:41 573:41 574:41 575:41 576:20 577:24 578:37 579:83 580:194 581:252 582:252 583:212 584:33 596:83 597:213 598:252 599:252 600:252 601:252 602:252 603:252 604:204 605:213 606:243 607:252 608:252 609:252 610:212 611:34 625:34 626:140 627:238 628:248 629:252 630:252 631:252 632:253 633:252 634:252 635:241 636:238 637:238 638:75 656:82 657:119 658:119 659:119 660:120 661:119 662:119 663:19 +1 127:20 128:254 129:255 130:37 155:19 156:253 157:253 158:134 183:19 184:253 185:253 186:246 187:125 211:76 212:253 213:253 214:253 215:158 239:207 240:253 241:253 242:253 243:158 267:207 268:253 269:253 270:253 271:158 294:48 295:223 296:253 297:253 298:243 299:106 322:141 323:253 324:253 325:253 326:113 349:65 350:237 351:253 352:253 353:253 354:36 377:76 378:253 379:253 380:253 381:253 382:36 405:76 406:253 407:253 408:253 409:253 410:36 433:76 434:253 435:253 436:253 437:118 438:4 460:4 461:148 462:253 463:253 464:253 465:103 488:10 489:253 490:253 491:253 492:253 493:103 516:10 517:253 518:253 519:253 520:173 521:7 544:10 545:253 546:253 547:253 548:168 572:143 573:253 574:253 575:239 576:49 600:198 601:253 602:253 603:234 615:140 628:198 629:253 630:253 631:234 656:198 657:253 658:253 659:234 +0 235:40 236:37 238:7 239:77 240:137 241:136 242:136 243:136 244:136 245:40 246:6 261:16 262:135 263:254 264:233 266:152 267:215 268:96 269:140 270:155 271:118 272:230 273:254 274:158 275:68 288:19 289:164 290:254 291:114 294:235 295:140 301:99 302:230 303:254 304:186 305:14 315:70 316:226 317:242 318:121 322:104 323:195 324:38 330:33 331:179 332:253 333:140 342:41 343:241 344:198 345:43 359:24 360:209 361:223 370:164 371:250 372:66 388:136 389:253 398:254 399:158 416:136 417:215 426:255 427:76 442:5 443:127 444:246 445:133 454:254 455:122 469:5 470:150 471:247 472:91 473:9 482:254 483:165 495:13 496:79 497:194 498:216 499:84 510:111 511:251 512:87 519:16 520:25 521:40 522:107 523:186 524:213 525:117 526:25 538:14 539:185 540:235 541:142 542:23 546:91 547:157 548:231 549:207 550:126 551:49 569:143 570:195 571:255 572:254 573:254 574:244 575:157 576:76 599:39 600:39 601:39 602:33 +1 128:166 129:255 130:187 131:6 156:165 157:253 158:253 159:13 183:15 184:191 185:253 186:253 187:13 211:49 212:253 213:253 214:253 215:13 239:141 240:253 241:253 242:169 243:4 266:4 267:189 268:253 269:249 270:53 294:69 295:253 296:253 297:246 322:69 323:253 324:253 325:246 350:118 351:253 352:253 353:124 378:206 379:253 380:231 381:21 405:66 406:241 407:253 408:199 433:105 434:253 435:253 436:89 460:3 461:228 462:253 463:252 464:86 488:111 489:253 490:253 491:205 516:166 517:253 518:253 519:75 543:43 544:249 545:253 546:193 547:9 570:4 571:160 572:253 573:253 574:184 598:37 599:253 600:253 601:253 602:88 626:140 627:253 628:253 629:186 630:18 654:14 655:253 656:253 657:27 +1 128:117 129:128 155:2 156:199 157:127 183:81 184:254 185:87 211:116 212:254 213:48 239:175 240:234 241:18 266:5 267:230 268:168 294:80 295:255 296:142 322:80 323:255 324:142 350:80 351:251 352:57 378:129 379:239 406:164 407:209 433:28 434:245 435:159 461:64 462:254 463:144 489:84 490:254 491:80 517:143 518:254 519:30 544:3 545:225 546:200 572:48 573:254 574:174 600:48 601:254 602:174 628:93 629:254 630:129 656:53 657:234 658:41 +1 129:159 130:142 156:11 157:220 158:141 184:78 185:254 186:141 212:111 213:254 214:109 240:196 241:221 242:15 267:26 268:221 269:159 295:63 296:254 297:159 323:178 324:254 325:93 350:7 351:191 352:254 353:97 378:42 379:255 380:254 381:41 406:42 407:254 408:195 409:10 434:141 435:255 436:78 461:11 462:202 463:254 464:59 489:86 490:254 491:254 492:59 517:142 518:254 519:248 520:52 545:142 546:254 547:195 573:142 574:254 575:164 601:142 602:254 603:77 629:142 630:254 631:131 657:77 658:172 659:5 +0 124:66 125:254 126:254 127:58 128:60 129:59 130:59 131:50 151:73 152:233 153:253 154:253 155:148 156:254 157:253 158:253 159:232 160:73 179:156 180:253 181:253 182:253 183:117 184:255 185:253 186:253 187:253 188:223 189:176 190:162 205:37 206:116 207:246 208:253 209:180 210:18 211:4 212:18 213:109 214:241 215:253 216:253 217:253 218:236 219:28 233:235 234:253 235:253 236:245 237:107 242:109 243:170 244:253 245:253 246:253 247:174 261:235 262:253 263:253 264:233 271:15 272:156 273:253 274:253 275:223 276:72 287:10 288:156 289:250 290:253 291:253 292:67 300:99 301:253 302:253 303:253 304:127 305:5 315:118 316:253 317:253 318:253 319:204 320:26 328:68 329:223 330:253 331:253 332:253 333:57 342:32 343:191 344:253 345:253 346:253 347:97 357:156 358:253 359:253 360:253 361:57 370:59 371:253 372:253 373:253 374:253 375:97 385:36 386:224 387:253 388:253 389:57 398:60 399:254 400:255 401:254 402:156 413:37 414:226 415:254 416:254 417:58 426:59 427:253 428:253 429:253 430:154 441:156 442:253 443:253 444:253 445:57 454:59 455:253 456:253 457:253 458:154 469:156 470:253 471:253 472:253 473:57 482:59 483:253 484:253 485:253 486:246 487:90 496:16 497:171 498:253 499:253 500:231 501:49 510:59 511:253 512:253 513:253 514:253 515:156 516:91 524:99 525:253 526:253 527:222 528:71 538:59 539:253 540:253 541:253 542:253 543:253 544:245 545:109 551:145 552:194 553:253 554:253 555:174 566:9 567:38 568:174 569:251 570:253 571:253 572:253 573:241 574:215 575:215 576:217 577:215 578:215 579:250 580:253 581:253 582:221 583:26 597:235 598:253 599:253 600:253 601:253 602:253 603:253 604:254 605:253 606:253 607:253 608:253 609:204 610:26 625:108 626:116 627:200 628:253 629:253 630:253 631:253 632:254 633:253 634:253 635:253 636:199 637:44 655:36 656:57 657:118 658:253 659:253 660:58 661:57 662:57 663:57 664:35 +1 129:101 130:222 131:84 157:225 158:252 159:84 184:89 185:246 186:208 187:19 212:128 213:252 214:195 239:79 240:253 241:252 242:195 267:141 268:255 269:253 270:133 294:26 295:240 296:253 297:252 298:55 322:60 323:252 324:253 325:154 326:12 349:7 350:178 351:252 352:253 353:27 377:57 378:252 379:252 380:253 381:27 405:57 406:253 407:253 408:204 409:15 433:104 434:252 435:252 436:94 460:19 461:209 462:252 463:252 488:101 489:252 490:252 491:157 516:225 517:252 518:252 519:112 544:226 545:253 546:240 547:63 572:225 573:252 574:223 600:225 601:252 602:223 628:225 629:252 630:242 631:75 656:146 657:252 658:236 659:50 +0 124:41 125:254 126:254 127:157 128:34 129:34 130:218 131:255 132:206 133:34 134:18 151:53 152:238 153:252 154:252 155:252 156:252 157:252 158:252 159:252 160:252 161:252 162:162 163:26 178:66 179:220 180:252 181:252 182:252 183:209 184:153 185:223 186:252 187:252 188:252 189:252 190:252 191:98 206:166 207:252 208:252 209:252 210:252 211:141 213:85 214:230 215:252 216:252 217:252 218:252 219:98 234:166 235:252 236:252 237:252 238:252 239:141 242:73 243:102 244:252 245:252 246:252 247:98 262:166 263:252 264:252 265:252 266:191 267:30 271:5 272:97 273:252 274:252 275:220 276:51 289:123 290:245 291:252 292:252 293:202 294:14 300:56 301:252 302:252 303:252 304:65 316:18 317:154 318:252 319:252 320:241 328:56 329:252 330:252 331:252 332:65 343:21 344:146 345:252 346:252 347:252 348:241 356:56 357:252 358:252 359:252 360:65 371:67 372:252 373:252 374:252 375:252 376:241 384:56 385:252 386:252 387:252 388:65 399:67 400:252 401:252 402:252 403:252 404:116 412:56 413:252 414:252 415:252 416:65 427:67 428:252 429:252 430:252 431:252 432:20 440:56 441:252 442:252 443:252 444:65 455:67 456:252 457:252 458:252 459:87 460:4 468:56 469:252 470:252 471:124 472:11 483:67 484:252 485:252 486:252 487:54 494:19 495:236 496:245 497:252 498:252 499:98 511:67 512:252 513:252 514:252 515:97 516:5 521:39 522:219 523:252 524:252 525:252 526:252 527:98 539:67 540:252 541:252 542:252 543:252 544:102 545:89 546:89 547:89 548:89 549:203 550:252 551:252 552:252 553:252 554:209 555:64 567:67 568:252 569:252 570:252 571:252 572:252 573:252 574:252 575:252 576:252 577:252 578:252 579:252 580:226 581:130 582:68 595:67 596:252 597:252 598:252 599:252 600:252 601:252 602:252 603:252 604:252 605:252 606:252 607:239 608:77 623:17 624:65 625:163 626:252 627:252 628:252 629:252 630:252 631:252 632:252 633:252 634:96 635:59 653:17 654:176 655:252 656:252 657:252 658:252 659:155 660:32 661:32 662:6 +0 96:56 97:247 98:121 124:24 125:242 126:245 127:122 153:231 154:253 155:253 156:104 157:12 181:90 182:253 183:253 184:254 185:221 186:120 187:120 188:85 206:67 207:75 208:36 209:11 210:56 211:222 212:254 213:253 214:253 215:253 216:245 217:207 218:36 233:86 234:245 235:249 236:105 239:44 240:224 241:230 242:253 243:253 244:253 245:253 246:214 247:10 260:8 261:191 262:253 263:143 269:29 270:119 271:119 272:158 273:253 274:253 275:94 288:15 289:253 290:226 291:48 300:4 301:183 302:253 303:248 304:56 316:42 317:253 318:178 329:179 330:253 331:184 332:14 344:164 345:253 346:178 357:179 358:253 359:163 371:61 372:254 373:254 374:179 384:76 385:254 386:254 387:164 399:60 400:253 401:253 402:178 411:29 412:206 413:253 414:253 415:40 427:60 428:253 429:253 430:178 439:120 440:253 441:253 442:245 443:13 455:60 456:253 457:253 458:178 467:120 468:253 469:239 470:63 483:60 484:253 485:253 486:178 494:14 495:238 496:253 497:179 511:18 512:190 513:253 514:231 515:70 521:43 522:184 523:253 524:253 525:74 540:86 541:253 542:253 543:239 544:134 545:8 548:56 549:163 550:253 551:253 552:213 553:35 568:16 569:253 570:253 571:253 572:253 573:240 574:239 575:239 576:247 577:253 578:253 579:210 580:27 596:4 597:59 598:204 599:253 600:253 601:253 602:253 603:253 604:254 605:253 606:250 607:110 626:31 627:122 628:253 629:253 630:253 631:253 632:255 633:217 634:98 +0 125:19 126:164 127:253 128:255 129:253 130:118 131:59 132:36 153:78 154:251 155:251 156:253 157:251 158:251 159:251 160:199 161:45 180:14 181:198 182:251 183:251 184:253 185:251 186:251 187:251 188:251 189:204 190:26 208:5 209:117 210:251 211:251 212:243 213:212 214:239 215:251 216:251 217:251 218:218 236:95 237:251 238:251 239:251 240:120 242:175 243:251 244:251 245:251 246:231 263:97 264:237 265:251 266:251 267:251 270:67 271:240 272:251 273:251 274:243 275:108 290:8 291:163 292:251 293:251 294:240 295:81 299:68 300:251 301:251 302:251 303:179 304:9 317:13 318:145 319:251 320:251 321:226 322:80 327:39 328:251 329:251 330:251 331:251 332:115 345:144 346:251 347:251 348:251 349:173 355:18 356:167 357:251 358:251 359:251 360:115 373:233 374:251 375:251 376:251 377:173 384:98 385:251 386:251 387:251 388:115 400:176 401:253 402:253 403:216 404:179 412:99 413:253 414:253 415:253 416:116 427:55 428:210 429:251 430:251 431:96 440:98 441:251 442:251 443:214 444:62 455:117 456:251 457:251 458:251 459:96 467:28 468:204 469:251 470:237 471:53 482:55 483:241 484:251 485:251 486:160 487:7 494:28 495:222 496:251 497:251 498:231 510:59 511:251 512:251 513:251 514:153 520:23 521:98 522:204 523:251 524:251 525:251 526:156 538:59 539:251 540:251 541:251 542:153 546:85 547:155 548:179 549:251 550:251 551:251 552:251 553:154 554:15 566:59 567:251 568:251 569:251 570:236 571:214 572:214 573:214 574:234 575:251 576:253 577:251 578:251 579:248 580:156 581:15 594:41 595:209 596:251 597:251 598:251 599:251 600:251 601:251 602:251 603:251 604:253 605:251 606:196 607:146 623:54 624:115 625:241 626:251 627:251 628:251 629:251 630:251 631:251 632:253 633:187 634:35 653:83 654:251 655:251 656:251 657:251 658:251 659:101 660:57 661:31 +1 129:232 130:255 131:107 156:58 157:244 158:253 159:106 184:95 185:253 186:253 187:106 212:95 213:253 214:253 215:106 240:95 241:253 242:249 243:69 268:144 269:253 270:192 295:97 296:233 297:253 298:66 323:195 324:253 325:253 326:5 350:38 351:232 352:253 353:182 354:2 377:10 378:160 379:253 380:231 381:53 405:42 406:253 407:253 408:158 433:141 434:253 435:253 436:115 460:75 461:245 462:253 463:183 464:4 487:1 488:147 489:253 490:251 491:58 515:20 516:253 517:253 518:180 543:202 544:253 545:226 546:27 571:243 572:253 573:212 598:85 599:251 600:253 601:173 626:209 627:253 628:244 629:57 654:169 655:253 656:174 +1 127:63 128:128 129:2 155:63 156:254 157:123 183:63 184:254 185:179 211:63 212:254 213:179 239:63 240:254 241:179 267:142 268:254 269:179 295:187 296:254 297:158 323:187 324:254 325:55 350:68 351:235 352:254 353:55 378:181 379:254 380:254 381:55 406:181 407:254 408:202 409:14 434:181 435:254 436:186 462:181 463:254 464:146 490:181 491:254 492:62 518:181 519:254 520:62 546:181 547:254 548:62 574:181 575:255 576:62 602:181 603:254 604:241 605:52 630:181 631:254 632:222 633:30 658:181 659:224 660:34 +1 130:131 131:255 132:184 133:15 157:99 158:247 159:253 160:182 161:15 185:124 186:253 187:253 188:253 189:38 212:9 213:171 214:253 215:253 216:140 217:1 240:47 241:253 242:253 243:251 244:117 267:43 268:219 269:253 270:253 271:153 295:78 296:253 297:253 298:253 299:84 323:97 324:253 325:253 326:244 327:74 350:69 351:243 352:253 353:253 354:183 377:10 378:168 379:253 380:253 381:215 382:34 405:31 406:253 407:253 408:253 409:129 433:107 434:253 435:253 436:242 437:67 460:24 461:204 462:253 463:253 464:187 488:95 489:253 490:253 491:201 492:25 516:239 517:253 518:253 519:176 543:119 544:251 545:253 546:253 547:138 570:30 571:212 572:253 573:252 574:165 575:8 598:193 599:253 600:253 601:222 626:193 627:253 628:253 629:189 654:193 655:253 656:201 657:27 +0 125:57 126:255 127:253 128:198 129:85 153:168 154:253 155:251 156:253 157:251 158:169 159:56 180:86 181:253 182:254 183:253 184:254 185:253 186:254 187:253 188:57 208:197 209:251 210:253 211:251 212:253 213:251 214:253 215:251 216:225 217:56 235:169 236:255 237:253 238:226 239:56 241:114 242:254 243:253 244:254 245:84 262:57 263:224 264:253 265:251 266:56 270:139 271:251 272:253 273:83 290:141 291:253 292:255 293:84 298:57 299:225 300:254 301:196 318:253 319:251 320:253 321:83 327:168 328:253 329:83 345:169 346:254 347:253 348:169 355:169 356:254 357:253 358:169 373:168 374:253 375:251 376:56 383:168 384:253 385:251 386:56 401:169 402:254 403:84 412:254 413:253 429:168 430:253 431:83 440:253 441:251 456:29 457:197 458:254 459:84 467:169 468:254 469:196 484:85 485:251 486:253 487:83 494:57 495:224 496:253 497:83 512:57 513:225 514:254 515:139 521:57 522:141 523:253 524:254 525:84 541:168 542:253 543:251 544:169 545:56 547:114 548:169 549:224 550:253 551:251 552:253 553:83 569:169 570:254 571:253 572:254 573:253 574:254 575:253 576:254 577:253 578:254 579:253 580:226 581:56 597:56 598:253 599:251 600:253 601:251 602:253 603:251 604:253 605:251 606:253 607:251 608:56 626:169 627:225 628:254 629:253 630:254 631:253 632:254 633:253 634:226 635:56 655:56 656:253 657:251 658:253 659:251 660:84 661:83 662:56 +0 127:12 128:105 129:224 130:255 131:247 132:22 155:131 156:254 157:254 158:243 159:252 160:76 182:131 183:225 184:254 185:224 186:48 187:136 208:13 209:109 210:252 211:254 212:254 213:254 214:197 215:76 235:9 236:181 237:254 238:254 239:240 240:229 241:237 242:254 243:252 244:152 245:21 262:9 263:143 264:254 265:254 266:226 267:36 269:22 270:138 271:254 272:254 273:188 289:13 290:181 291:254 292:254 293:250 294:64 298:2 299:53 300:236 301:252 302:131 317:102 318:254 319:254 320:254 321:111 328:56 329:243 330:251 331:42 344:30 345:186 346:254 347:254 348:206 349:29 357:199 358:254 359:91 372:92 373:254 374:254 375:237 376:13 385:134 386:254 387:91 400:133 401:254 402:254 403:126 413:134 414:250 415:17 428:187 429:254 430:237 431:23 441:200 442:183 456:187 457:254 458:213 467:2 468:134 469:252 470:101 484:183 485:254 486:133 495:14 496:254 497:234 498:34 512:92 513:254 514:161 522:84 523:204 524:254 525:56 540:92 541:254 542:229 549:85 550:252 551:252 552:188 553:11 568:56 569:252 570:229 575:3 576:53 577:235 578:253 579:166 597:224 598:245 599:130 600:68 601:68 602:134 603:214 604:254 605:254 606:159 625:141 626:254 627:254 628:254 629:254 630:254 631:254 632:233 633:95 634:3 653:14 654:152 655:254 656:254 657:254 658:186 659:157 660:53 +1 130:226 131:247 132:55 157:99 158:248 159:254 160:230 161:30 185:125 186:254 187:254 188:254 189:38 213:125 214:254 215:254 216:212 217:24 240:18 241:223 242:254 243:252 244:118 268:24 269:254 270:254 271:239 295:27 296:195 297:254 298:254 299:93 323:78 324:254 325:254 326:246 327:74 351:158 352:254 353:254 354:185 378:41 379:239 380:254 381:254 382:43 405:22 406:218 407:254 408:254 409:167 410:9 433:32 434:254 435:254 436:254 437:130 460:24 461:187 462:254 463:254 464:234 465:16 488:189 489:254 490:254 491:254 492:128 515:64 516:247 517:254 518:255 519:219 520:42 543:139 544:254 545:254 546:222 547:40 570:30 571:213 572:254 573:235 574:45 598:194 599:254 600:254 601:223 626:194 627:254 628:254 629:190 654:194 655:254 656:202 657:27 +1 130:166 131:253 132:124 133:53 158:140 159:251 160:251 161:180 185:125 186:246 187:251 188:251 189:51 212:32 213:190 214:251 215:251 216:251 217:103 240:21 241:174 242:251 243:251 244:251 268:73 269:176 270:253 271:253 272:201 296:149 297:251 298:251 299:251 300:71 323:27 324:228 325:251 326:251 327:157 328:10 351:180 352:253 353:251 354:251 355:142 377:27 378:180 379:231 380:253 381:251 382:96 383:41 405:89 406:253 407:253 408:255 409:211 410:25 433:217 434:251 435:251 436:253 437:107 460:21 461:221 462:251 463:251 464:242 465:92 487:32 488:190 489:251 490:251 491:251 492:103 515:202 516:251 517:251 518:251 519:122 542:53 543:255 544:253 545:253 546:221 547:51 570:180 571:253 572:251 573:251 574:142 598:180 599:253 600:251 601:251 602:142 626:180 627:253 628:251 629:157 630:82 654:180 655:253 656:147 657:10 +1 129:17 130:206 131:229 132:44 157:2 158:125 159:254 160:123 185:95 186:254 187:254 188:123 212:78 213:240 214:254 215:254 216:123 240:100 241:254 242:254 243:254 244:123 267:2 268:129 269:254 270:254 271:220 272:20 295:9 296:254 297:254 298:254 299:123 322:22 323:179 324:254 325:254 326:254 327:49 350:83 351:254 352:254 353:254 354:183 355:19 378:136 379:254 380:254 381:254 382:139 404:3 405:111 406:252 407:254 408:254 409:232 410:45 432:67 433:254 434:254 435:254 436:216 437:40 459:14 460:192 461:254 462:254 463:254 464:140 486:23 487:192 488:254 489:254 490:254 491:246 514:77 515:254 516:254 517:255 518:241 519:100 541:65 542:235 543:254 544:254 545:254 546:172 568:30 569:238 570:254 571:254 572:254 573:219 574:26 596:34 597:254 598:254 599:254 600:216 601:41 624:34 625:254 626:254 627:254 628:188 652:12 653:170 654:254 655:254 656:82 +1 130:218 131:253 132:124 157:84 158:236 159:251 160:251 184:63 185:236 186:251 187:251 188:122 212:73 213:251 214:251 215:251 216:173 240:202 241:251 242:251 243:251 244:71 267:53 268:255 269:253 270:253 271:253 272:72 295:180 296:253 297:251 298:251 299:188 300:30 323:180 324:253 325:251 326:251 327:142 350:47 351:211 352:253 353:251 354:235 355:82 377:27 378:211 379:251 380:253 381:251 382:215 405:89 406:253 407:253 408:255 409:253 410:164 433:217 434:251 435:251 436:253 437:168 438:15 460:21 461:221 462:251 463:251 464:253 465:107 487:32 488:190 489:251 490:251 491:251 492:221 493:61 515:73 516:251 517:251 518:251 519:251 520:180 543:255 544:253 545:253 546:253 547:201 570:105 571:253 572:251 573:251 574:251 575:71 598:180 599:253 600:251 601:246 602:137 603:10 626:180 627:253 628:251 629:215 654:180 655:253 656:251 657:86 +1 124:102 125:180 126:1 152:140 153:254 154:130 180:140 181:254 182:204 208:140 209:254 210:204 236:72 237:254 238:204 264:25 265:231 266:250 267:135 292:11 293:211 294:254 295:222 321:101 322:254 323:250 324:15 349:96 350:254 351:254 352:95 377:2 378:251 379:254 380:95 405:2 406:251 407:254 408:95 433:96 434:254 435:254 436:95 461:53 462:253 463:254 464:139 490:250 491:254 492:235 493:27 518:201 519:254 520:254 521:128 546:80 547:254 548:254 549:139 574:65 575:254 576:254 577:139 602:150 603:254 604:254 605:139 630:229 631:254 632:254 633:43 658:52 659:196 660:168 661:9 +0 128:87 129:208 130:249 155:27 156:212 157:254 158:195 182:118 183:225 184:254 185:254 186:232 187:147 188:46 209:115 210:248 211:254 212:254 213:254 214:254 215:254 216:230 217:148 218:12 236:18 237:250 238:254 239:245 240:226 241:254 242:254 243:254 244:254 245:254 246:148 263:92 264:205 265:254 266:250 267:101 268:20 269:194 270:254 271:254 272:254 273:254 274:229 275:53 291:152 292:254 293:254 294:94 297:14 298:124 299:187 300:254 301:254 302:254 303:213 318:95 319:252 320:254 321:206 322:15 327:3 328:6 329:51 330:231 331:254 332:94 345:50 346:246 347:254 348:254 349:20 358:200 359:254 360:96 372:21 373:184 374:254 375:254 376:147 377:2 386:200 387:254 388:96 400:177 401:254 402:254 403:218 404:33 413:16 414:211 415:254 416:96 427:11 428:219 429:254 430:251 431:92 441:84 442:254 443:232 444:44 455:101 456:254 457:254 458:141 469:162 470:254 471:231 472:42 483:235 484:254 485:227 486:42 496:51 497:238 498:254 499:213 511:235 512:254 513:199 524:160 525:254 526:229 527:52 539:235 540:254 541:199 549:10 550:84 551:150 552:253 553:254 554:147 567:235 568:254 569:213 570:20 575:17 576:63 577:158 578:254 579:254 580:254 581:155 582:12 595:122 596:248 597:254 598:204 599:98 600:42 601:177 602:180 603:200 604:254 605:254 606:253 607:213 608:82 609:10 624:203 625:254 626:254 627:254 628:254 629:254 630:254 631:254 632:251 633:219 634:94 652:35 653:221 654:254 655:254 656:254 657:254 658:254 659:217 660:95 +1 126:134 127:230 154:133 155:231 156:10 182:133 183:253 184:96 210:133 211:253 212:96 238:133 239:253 240:183 266:133 267:253 268:217 294:133 295:253 296:217 322:133 323:253 324:217 350:133 351:253 352:217 378:133 379:253 380:217 406:134 407:254 408:218 434:133 435:253 436:159 462:133 463:253 464:199 490:156 491:253 492:96 518:254 519:247 520:73 546:254 547:248 548:74 573:99 574:254 575:245 576:64 600:89 601:230 602:254 603:125 627:140 628:251 629:253 630:243 631:10 655:114 656:242 657:195 658:69 +1 125:29 126:85 127:255 128:139 153:197 154:251 155:253 156:251 181:254 182:253 183:254 184:253 209:253 210:251 211:253 212:251 237:254 238:253 239:254 240:253 265:253 266:251 267:253 268:138 293:254 294:253 295:254 296:196 321:253 322:251 323:253 324:196 349:254 350:253 351:254 352:84 377:253 378:251 379:253 380:196 405:254 406:253 407:254 408:253 433:253 434:251 435:253 436:251 461:254 462:253 463:254 464:253 489:253 490:251 491:253 492:251 517:254 518:253 519:254 520:253 545:253 546:251 547:253 548:251 573:254 574:253 575:254 576:253 601:253 602:251 603:253 604:251 629:57 630:225 631:254 632:253 658:56 659:253 660:251 +1 125:149 126:255 127:254 128:58 153:215 154:253 155:183 156:2 180:41 181:232 182:253 183:181 208:92 209:253 210:253 211:181 236:92 237:253 238:253 239:181 264:92 265:253 266:253 267:181 292:92 293:253 294:253 295:181 320:92 321:253 322:253 323:181 348:92 349:253 350:253 351:181 376:92 377:253 378:253 379:181 404:92 405:253 406:253 407:181 432:92 433:253 434:253 435:181 460:92 461:253 462:253 463:181 488:31 489:228 490:253 491:181 517:198 518:253 519:228 520:54 545:33 546:226 547:253 548:195 549:7 574:199 575:253 576:253 577:75 602:34 603:218 604:253 605:228 606:117 607:14 608:12 631:33 632:219 633:253 634:253 635:253 636:211 660:32 661:123 662:149 663:230 664:41 +1 130:79 131:203 132:141 157:51 158:240 159:240 160:140 185:88 186:252 187:252 188:140 213:197 214:252 215:252 216:140 241:197 242:252 243:252 244:140 268:147 269:253 270:253 271:253 295:38 296:234 297:252 298:242 299:89 323:113 324:252 325:252 326:223 350:16 351:207 352:252 353:252 354:129 377:16 378:203 379:253 380:252 381:220 382:37 405:29 406:253 407:255 408:253 409:56 432:19 433:181 434:252 435:253 436:176 437:6 460:166 461:252 462:252 463:228 464:52 487:10 488:203 489:252 490:252 491:126 514:63 515:178 516:252 517:252 518:173 542:114 543:253 544:253 545:225 570:238 571:252 572:252 573:99 596:7 597:135 598:253 599:252 600:176 601:19 624:29 625:252 626:253 627:252 628:55 652:13 653:189 654:253 655:204 656:25 +1 126:94 127:254 128:75 154:166 155:253 156:231 182:208 183:253 184:147 210:208 211:253 212:116 238:208 239:253 240:168 266:146 267:254 268:222 294:166 295:253 296:116 322:208 323:253 324:116 350:166 351:253 352:158 378:145 379:253 380:231 406:209 407:254 408:169 434:187 435:253 436:168 462:93 463:253 464:116 490:93 491:253 492:116 518:93 519:253 520:116 546:94 547:254 548:179 549:11 574:93 575:253 576:246 577:101 602:145 603:253 604:255 605:92 630:93 631:253 632:246 633:59 658:93 659:253 660:74 +0 127:46 128:105 129:254 130:254 131:224 132:59 133:59 134:9 155:196 156:254 157:253 158:253 159:253 160:253 161:253 162:128 182:96 183:235 184:254 185:253 186:253 187:253 188:253 189:253 190:247 191:122 208:4 209:101 210:244 211:253 212:254 213:234 214:241 215:253 216:253 217:253 218:253 219:186 220:18 236:96 237:253 238:253 239:253 240:232 241:83 242:109 243:170 244:253 245:253 246:253 247:253 248:116 264:215 265:253 266:253 267:253 268:196 271:40 272:253 273:253 274:253 275:253 276:116 290:8 291:141 292:247 293:253 294:253 295:237 296:29 299:6 300:38 301:171 302:253 303:253 304:116 317:13 318:146 319:253 320:253 321:253 322:253 323:57 329:156 330:253 331:253 332:116 345:40 346:253 347:253 348:253 349:253 350:178 351:27 357:156 358:253 359:253 360:116 372:136 373:204 374:253 375:253 376:253 377:192 378:27 385:156 386:253 387:253 388:116 399:28 400:195 401:254 402:254 403:254 404:250 405:135 412:99 413:255 414:254 415:254 416:117 427:118 428:253 429:253 430:253 431:253 432:142 439:19 440:170 441:253 442:253 443:216 444:62 454:42 455:212 456:253 457:253 458:253 459:253 460:38 466:124 467:188 468:253 469:253 470:253 471:174 482:59 483:253 484:253 485:253 486:237 487:93 488:3 491:31 492:40 493:130 494:247 495:253 496:253 497:253 498:204 499:13 510:59 511:253 512:253 513:253 514:154 518:54 519:218 520:254 521:253 522:253 523:253 524:253 525:253 526:38 538:59 539:253 540:253 541:253 542:215 543:156 544:156 545:156 546:209 547:253 548:255 549:253 550:253 551:253 552:192 553:97 554:15 566:55 567:242 568:253 569:253 570:253 571:253 572:253 573:253 574:253 575:253 576:254 577:253 578:253 579:204 580:23 595:118 596:253 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:254 605:216 606:174 607:13 623:54 624:116 625:243 626:253 627:253 628:253 629:253 630:253 631:146 632:117 633:62 653:53 654:132 655:253 656:253 657:192 658:57 659:13 +1 125:42 126:232 127:254 128:58 153:86 154:253 155:253 156:58 181:86 182:253 183:253 184:58 209:206 210:253 211:253 212:58 237:215 238:253 239:253 240:58 265:215 266:253 267:253 268:58 293:215 294:253 295:253 296:58 321:215 322:253 323:253 324:58 349:215 350:253 351:253 352:58 377:215 378:253 379:253 380:58 405:215 406:253 407:253 408:58 433:188 434:253 435:253 436:85 461:86 462:253 463:253 464:200 465:12 489:29 490:223 491:253 492:253 493:151 518:209 519:253 520:253 521:194 546:128 547:253 548:253 549:200 550:8 574:32 575:213 576:253 577:253 578:152 579:6 603:32 604:221 605:253 606:253 607:153 608:5 632:90 633:215 634:253 635:253 636:151 661:59 662:253 663:253 664:84 +1 156:60 157:229 158:38 184:187 185:254 186:78 211:121 212:252 213:254 214:78 239:197 240:254 241:206 242:6 267:197 268:254 269:202 294:27 295:218 296:233 297:62 322:117 323:254 324:195 350:203 351:254 352:195 377:64 378:244 379:254 380:195 405:79 406:254 407:255 408:161 433:79 434:254 435:254 436:65 461:79 462:254 463:241 464:52 489:79 490:254 491:189 517:79 518:254 519:189 545:79 546:254 547:189 573:79 574:254 575:189 601:79 602:254 603:194 604:5 629:35 630:219 631:254 632:72 658:34 659:223 660:195 687:129 688:195 +1 101:11 102:150 103:72 129:37 130:251 131:71 157:63 158:251 159:71 185:217 186:251 187:71 213:217 214:251 215:71 240:145 241:253 242:253 243:72 267:42 268:206 269:251 270:251 271:71 295:99 296:251 297:251 298:251 299:71 323:253 324:251 325:251 326:251 327:71 350:130 351:253 352:251 353:251 354:251 355:71 377:110 378:253 379:255 380:253 381:253 382:253 383:72 405:109 406:251 407:253 408:251 409:251 410:188 411:30 433:109 434:251 435:253 436:251 437:246 438:123 460:16 461:170 462:251 463:253 464:251 465:215 488:37 489:251 490:251 491:253 492:251 493:86 516:218 517:253 518:253 519:255 520:253 521:35 543:84 544:236 545:251 546:251 547:253 548:168 549:15 571:144 572:251 573:251 574:251 575:190 576:15 599:144 600:251 601:251 602:251 603:180 626:53 627:221 628:251 629:251 630:251 631:180 +0 127:45 128:254 129:254 130:254 131:148 132:24 133:9 154:43 155:254 156:252 157:252 158:252 159:252 160:252 161:121 162:13 181:58 182:237 183:254 184:252 185:252 186:252 187:252 188:252 189:252 190:68 208:69 209:224 210:252 211:254 212:252 213:252 214:252 215:252 216:252 217:252 218:135 219:17 235:75 236:216 237:252 238:252 239:254 240:231 241:168 242:252 243:252 244:252 245:252 246:252 247:45 262:77 263:212 264:252 265:252 266:252 267:242 268:93 269:32 270:114 271:177 272:252 273:252 274:252 275:158 276:12 289:75 290:212 291:252 292:252 293:252 294:252 295:231 299:116 300:252 301:252 302:252 303:252 304:21 316:69 317:216 318:252 319:252 320:252 321:252 322:252 323:62 327:116 328:252 329:252 330:252 331:252 332:21 344:93 345:252 346:252 347:252 348:252 349:252 350:62 355:21 356:158 357:252 358:252 359:252 360:21 371:64 372:239 373:252 374:252 375:252 376:252 377:252 378:21 384:139 385:252 386:252 387:252 388:21 398:5 399:87 400:254 401:254 402:254 403:254 404:237 405:41 411:11 412:150 413:254 414:254 415:254 416:22 425:5 426:85 427:252 428:252 429:252 430:252 431:222 432:55 439:116 440:252 441:252 442:252 443:214 444:18 453:24 454:252 455:252 456:252 457:252 458:252 459:91 466:26 467:153 468:252 469:252 470:252 471:45 481:24 482:252 483:252 484:252 485:252 486:252 487:91 492:18 493:93 494:151 495:252 496:252 497:252 498:184 499:28 509:24 510:252 511:252 512:252 513:252 514:252 515:164 516:116 517:116 518:116 519:117 520:141 521:252 522:252 523:252 524:252 525:252 526:68 537:24 538:252 539:252 540:252 541:252 542:252 543:252 544:252 545:252 546:252 547:254 548:252 549:252 550:252 551:252 552:252 553:163 554:31 565:9 566:121 567:252 568:252 569:252 570:252 571:252 572:252 573:252 574:252 575:254 576:252 577:252 578:252 579:178 580:91 581:33 594:13 595:119 596:252 597:252 598:252 599:252 600:252 601:252 602:252 603:254 604:252 605:252 606:184 607:37 623:13 624:121 625:252 626:252 627:252 628:252 629:252 630:252 631:254 632:214 633:45 634:28 652:8 653:21 654:21 655:169 656:252 657:252 658:41 659:22 660:18 +0 125:218 126:253 127:253 128:255 129:149 130:62 151:42 152:144 153:236 154:251 155:251 156:253 157:251 158:236 159:144 160:144 179:99 180:251 181:251 182:251 183:225 184:253 185:251 186:251 187:251 188:251 189:166 190:16 206:79 207:253 208:251 209:251 210:204 211:41 212:143 213:205 214:251 215:251 216:251 217:253 218:169 219:15 233:79 234:231 235:253 236:251 237:225 238:41 241:41 242:226 243:251 244:251 245:253 246:251 247:164 260:37 261:253 262:253 263:255 264:253 265:35 271:79 272:232 273:255 274:253 275:227 276:42 288:140 289:251 290:251 291:253 292:168 293:15 300:77 301:253 302:251 303:251 304:142 315:21 316:221 317:251 318:251 319:164 320:15 329:227 330:251 331:251 332:236 333:61 342:32 343:190 344:251 345:251 346:251 357:73 358:251 359:251 360:251 361:71 370:73 371:251 372:251 373:251 374:251 385:73 386:251 387:251 388:251 389:71 398:73 399:253 400:253 401:253 402:201 413:73 414:253 415:253 416:253 417:72 426:176 427:251 428:251 429:251 430:71 441:73 442:251 443:251 444:251 445:71 454:253 455:251 456:251 457:157 458:10 469:73 470:251 471:251 472:251 473:71 482:253 483:251 484:251 485:142 497:150 498:251 499:251 500:204 501:41 510:124 511:251 512:251 513:220 514:180 524:130 525:253 526:251 527:225 528:41 538:73 539:253 540:253 541:253 542:253 543:73 544:73 545:10 549:42 550:73 551:150 552:253 553:255 554:253 555:216 566:31 567:189 568:251 569:251 570:251 571:253 572:251 573:159 574:144 575:144 576:145 577:206 578:251 579:251 580:251 581:253 582:168 583:92 595:20 596:195 597:251 598:251 599:253 600:251 601:251 602:251 603:251 604:253 605:251 606:251 607:251 608:225 609:164 610:15 624:21 625:142 626:220 627:253 628:251 629:251 630:251 631:251 632:253 633:251 634:251 635:204 636:41 654:51 655:72 656:174 657:251 658:251 659:251 660:253 661:147 662:71 663:41 +0 127:60 128:96 129:96 130:48 153:16 154:171 155:228 156:253 157:251 158:220 159:51 160:32 181:127 182:251 183:251 184:253 185:251 186:251 187:251 188:251 189:80 207:24 208:182 209:236 210:251 211:211 212:189 213:236 214:251 215:251 216:251 217:242 218:193 234:100 235:194 236:251 237:251 238:211 239:35 241:71 242:173 243:251 244:251 245:253 246:240 247:158 248:19 261:64 262:253 263:255 264:253 265:205 266:19 271:40 272:218 273:255 274:253 275:253 276:91 288:16 289:186 290:251 291:253 292:247 293:110 300:39 301:233 302:251 303:251 304:188 315:16 316:189 317:251 318:251 319:205 320:110 329:48 330:220 331:251 332:220 333:48 343:72 344:251 345:251 346:251 347:158 358:51 359:251 360:251 361:232 371:190 372:251 373:251 374:251 375:59 386:32 387:251 388:251 389:251 398:96 399:253 400:253 401:253 402:95 414:32 415:253 416:253 417:193 426:214 427:251 428:251 429:204 430:23 442:52 443:251 444:251 445:94 454:253 455:251 456:251 457:109 469:48 470:221 471:251 472:219 473:47 482:253 483:251 484:251 485:70 497:234 498:251 499:251 500:188 510:253 511:251 512:251 513:188 523:40 524:158 525:253 526:251 527:172 528:70 539:191 540:253 541:253 542:253 543:96 544:24 549:12 550:174 551:253 552:253 553:255 554:221 567:71 568:251 569:251 570:251 571:253 572:205 573:190 574:190 575:190 576:191 577:197 578:251 579:251 580:231 581:221 582:93 595:16 596:126 597:236 598:251 599:253 600:251 601:251 602:251 603:251 604:253 605:251 606:251 607:140 608:47 625:67 626:188 627:189 628:188 629:188 630:188 631:188 632:189 633:188 634:109 635:4 +0 126:32 127:202 128:255 129:253 130:253 131:175 132:21 152:84 153:144 154:190 155:251 156:253 157:251 158:251 159:251 160:174 176:6 177:37 178:166 179:218 180:236 181:251 182:251 183:251 184:253 185:251 186:251 187:251 188:251 189:156 204:115 205:251 206:251 207:253 208:251 209:251 210:251 211:251 212:253 213:251 214:251 215:251 216:251 217:180 231:105 232:241 233:251 234:251 235:253 236:251 237:251 238:251 239:122 240:72 241:71 242:71 243:148 244:251 245:180 258:73 259:253 260:253 261:253 262:253 263:202 264:253 265:253 266:143 286:31 287:189 288:251 289:251 290:251 291:31 292:189 293:251 294:142 314:63 315:236 316:251 317:251 318:96 320:124 321:246 322:142 330:21 331:166 332:21 342:73 343:251 344:251 345:251 346:71 349:217 350:142 357:32 358:190 359:251 360:142 370:73 371:251 372:251 373:251 374:71 377:217 378:142 385:73 386:251 387:251 388:142 398:73 399:253 400:253 401:253 402:72 405:156 406:103 413:73 414:253 415:253 416:253 417:72 426:73 427:251 428:251 429:251 430:174 441:73 442:251 443:251 444:251 445:71 454:73 455:251 456:251 457:251 458:251 469:73 470:251 471:251 472:251 473:71 482:42 483:205 484:251 485:251 486:251 487:79 497:73 498:251 499:251 500:251 501:71 511:41 512:226 513:251 514:251 515:232 516:77 525:73 526:251 527:251 528:251 529:71 540:166 541:253 542:253 543:255 544:253 545:227 546:73 547:21 553:125 554:253 555:253 556:143 568:16 569:169 570:251 571:253 572:251 573:251 574:251 575:174 576:105 579:63 580:144 581:253 582:251 583:251 584:142 597:15 598:35 599:253 600:251 601:251 602:251 603:251 604:243 605:217 606:217 607:231 608:251 609:253 610:251 611:220 612:20 627:143 628:142 629:236 630:251 631:251 632:253 633:251 634:251 635:251 636:251 637:253 638:251 639:137 657:61 658:71 659:200 660:253 661:251 662:251 663:251 664:251 665:201 666:71 667:10 +1 130:218 131:170 132:108 157:32 158:227 159:252 160:232 185:129 186:252 187:252 188:252 212:1 213:253 214:252 215:252 216:168 240:144 241:253 242:252 243:236 244:62 268:144 269:253 270:252 271:215 296:144 297:253 298:252 299:112 323:21 324:206 325:253 326:252 327:71 351:99 352:253 353:255 354:119 378:63 379:242 380:252 381:253 382:35 406:94 407:252 408:252 409:154 410:10 433:145 434:237 435:252 436:252 461:255 462:253 463:253 464:108 487:11 488:155 489:253 490:252 491:179 492:15 514:11 515:150 516:252 517:253 518:200 519:20 542:73 543:252 544:252 545:253 546:97 569:47 570:233 571:253 572:253 596:1 597:149 598:252 599:252 600:252 624:1 625:252 626:252 627:246 628:132 652:1 653:169 654:252 655:132 +1 130:116 131:255 132:123 157:29 158:213 159:253 160:122 185:189 186:253 187:253 188:122 213:189 214:253 215:253 216:122 241:189 242:253 243:253 244:122 267:2 268:114 269:243 270:253 271:186 272:19 295:100 296:253 297:253 298:253 299:48 323:172 324:253 325:253 326:253 327:48 351:172 352:253 353:253 354:182 355:19 378:133 379:251 380:253 381:175 382:4 405:107 406:251 407:253 408:253 409:65 432:26 433:194 434:253 435:253 436:214 437:40 459:105 460:205 461:253 462:253 463:125 464:40 487:139 488:253 489:253 490:253 491:81 514:41 515:231 516:253 517:253 518:159 519:16 541:65 542:155 543:253 544:253 545:172 546:4 569:124 570:253 571:253 572:253 573:98 597:124 598:253 599:253 600:214 601:41 624:22 625:207 626:253 627:253 628:139 653:124 654:253 655:162 656:9 From 745e496c59cfece2fcd6120ecc366dcab07b293a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 22 Apr 2014 14:27:49 -0700 Subject: [PATCH 131/641] [Fix #204] Eliminate delay between binding and log checking **Bug**: In the existing history server, there is a `spark.history.updateInterval` seconds delay before application logs show up on the UI. **Cause**: This is because the following events happen in this order: (1) The background thread that checks for logs starts, but realizes the server has not yet bound and so waits for N seconds, (2) server binds, (3) N seconds later the background thread finds that the server has finally bound to a port, and so finally checks for application logs. **Fix**: This PR forces the log checking thread to start immediately after binding. It also documents two relevant environment variables that are currently missing. Author: Andrew Or Closes #441 from andrewor14/history-server-fix and squashes the following commits: b2eb46e [Andrew Or] Document SPARK_PUBLIC_DNS and SPARK_HISTORY_OPTS for the history server e8d1fbc [Andrew Or] Eliminate delay between binding and checking for logs --- .../spark/deploy/history/HistoryServer.scala | 5 +++++ docs/monitoring.md | 19 +++++++++++++++---- 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index cf64700f9098c..b8f56234d37ec 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -98,6 +98,11 @@ class HistoryServer( def initialize() { attachPage(new HistoryPage(this)) attachHandler(createStaticHandler(STATIC_RESOURCE_DIR, "/static")) + } + + /** Bind to the HTTP server behind this web interface. */ + override def bind() { + super.bind() logCheckingThread.start() } diff --git a/docs/monitoring.md b/docs/monitoring.md index 4c91c3a5929bf..144be3daf1208 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -39,22 +39,33 @@ You can start a the history server by executing: The base logging directory must be supplied, and should contain sub-directories that each represents an application's event logs. This creates a web interface at -`http://:18080` by default. The history server depends on the following variables: +`http://:18080` by default. The history server can be configured as follows: - + + + + + + + + +
      Environment VariableMeaning
      SPARK_DAEMON_MEMORYMemory to allocate to the history server. (default: 512m).Memory to allocate to the history server (default: 512m).
      SPARK_DAEMON_JAVA_OPTS JVM options for the history server (default: none).
      SPARK_PUBLIC_DNS + The public address for the history server. If this is not set, links to application history + may use the internal address of the server, resulting in broken links (default: none). +
      SPARK_HISTORY_OPTS + spark.history.* configuration options for the history server (default: none). +
      -Further, the history server can be configured as follows: - From 0ea0b1a2d66044af97ab84bf3014ff8dfe91fee4 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 22 Apr 2014 14:28:41 -0700 Subject: [PATCH 132/641] Fix compilation on Hadoop 2.4.x. Author: Marcelo Vanzin Closes #483 from vanzin/yarn-2.4 and squashes the following commits: 0fc57d8 [Marcelo Vanzin] Fix compilation on Hadoop 2.4.x. --- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 c00b63669ca8e..d264ecbeaf7d8 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 @@ -458,7 +458,8 @@ object ClientBase { } /** Add entry to the classpath. */ - def addClasspathEntry(path: String) = Apps.addToEnvironment(env, Environment.CLASSPATH.name, path) + def addClasspathEntry(path: String) = YarnSparkHadoopUtil.addToEnvironment(env, + Environment.CLASSPATH.name, path, File.pathSeparator) /** Add entry to the classpath. Interpreted as a path relative to the working directory. */ def addPwdClasspathEntry(entry: String) = addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + entry) From ea8cea82a02099bb66f1e77b757e4d96cc31d6e2 Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Tue, 22 Apr 2014 15:05:12 -0700 Subject: [PATCH 133/641] [SPARK-1570] Fix classloading in JavaSQLContext.applySchema I think I hit a class loading issue when running JavaSparkSQL example using spark-submit in local mode. Author: Kan Zhang Closes #484 from kanzhang/SPARK-1570 and squashes the following commits: feaaeba [Kan Zhang] [SPARK-1570] Fix classloading in JavaSQLContext.applySchema --- .../org/apache/spark/sql/api/java/JavaSQLContext.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 26922f7f336e2..a7347088794a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} +import org.apache.spark.util.Utils /** * The entry point for executing Spark SQL queries from a Java program. @@ -84,10 +85,11 @@ class JavaSQLContext(sparkContext: JavaSparkContext) { */ def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): JavaSchemaRDD = { val schema = getSchema(beanClass) - val className = beanClass.getCanonicalName + val className = beanClass.getName val rowRdd = rdd.rdd.mapPartitions { iter => // BeanInfo is not serializable so we must rediscover it remotely for each partition. - val localBeanInfo = Introspector.getBeanInfo(Class.forName(className)) + val localBeanInfo = Introspector.getBeanInfo( + Class.forName(className, true, Utils.getContextOrSparkClassLoader)) val extractors = localBeanInfo.getPropertyDescriptors.filterNot(_.getName == "class").map(_.getReadMethod) From 995fdc96bcd2c540804401eaab009a777d7d7aa9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 22 Apr 2014 19:22:06 -0700 Subject: [PATCH 134/641] Assorted clean-up for Spark-on-YARN. In particular when the HADOOP_CONF_DIR is not not specified. Author: Patrick Wendell Closes #488 from pwendell/hadoop-cleanup and squashes the following commits: fe95f13 [Patrick Wendell] Changes based on Andrew's feeback 18d09c1 [Patrick Wendell] Review comments from Andrew 17929cc [Patrick Wendell] Assorted clean-up for Spark-on-YARN. --- conf/spark-env.sh.template | 2 ++ .../org/apache/spark/deploy/SparkSubmitArguments.scala | 9 +++++++++ docs/hadoop-third-party-distributions.md | 9 ++------- .../scala/org/apache/spark/deploy/yarn/ClientBase.scala | 8 +++++--- 4 files changed, 18 insertions(+), 10 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 177a21cc0377f..f906be611a931 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -5,6 +5,7 @@ # Options read when launching programs locally with # ./bin/run-example or ./bin/spark-submit +# - HADOOP_CONF_DIR, to point Spark towards Hadoop configuration files # - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node # - SPARK_PUBLIC_DNS, to set the public dns name of the driver program # - SPARK_CLASSPATH, default classpath entries to append @@ -17,6 +18,7 @@ # - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos # Options read in YARN client mode +# - HADOOP_CONF_DIR, to point Spark towards Hadoop configuration files # - SPARK_EXECUTOR_INSTANCES, Number of workers to start (Default: 2) # - SPARK_EXECUTOR_CORES, Number of cores for the workers (Default: 1). # - SPARK_EXECUTOR_MEMORY, Memory per Worker (e.g. 1000M, 2G) (Default: 1G) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 02502adfbd0c4..cc976565cc72f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -116,6 +116,15 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { if (args.length == 0) printUsageAndExit(-1) if (primaryResource == null) SparkSubmit.printErrorAndExit("Must specify a primary resource") if (mainClass == null) SparkSubmit.printErrorAndExit("Must specify a main class with --class") + + if (master.startsWith("yarn")) { + val hasHadoopEnv = sys.env.contains("HADOOP_CONF_DIR") || sys.env.contains("YARN_CONF_DIR") + val testing = sys.env.contains("SPARK_TESTING") + if (!hasHadoopEnv && !testing) { + throw new Exception(s"When running with master '$master' " + + "either HADOOP_CONF_DIR or YARN_CONF_DIR must be set in the environment.") + } + } } override def toString = { diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index de6a2b0a43bd5..454877a7fa8a5 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -110,10 +110,5 @@ The location of these configuration files varies across CDH and HDP versions, bu a common location is inside of `/etc/hadoop/conf`. Some tools, such as Cloudera Manager, create configurations on-the-fly, but offer a mechanisms to download copies of them. -There are a few ways to make these files visible to Spark: - -* You can copy these files into `$SPARK_HOME/conf` and they will be included in Spark's -classpath automatically. -* If you are running Spark on the same nodes as Hadoop _and_ your distribution includes both -`hdfs-site.xml` and `core-site.xml` in the same directory, you can set `HADOOP_CONF_DIR` -in `$SPARK_HOME/spark-env.sh` to that directory. +To make these files visible to Spark, set `HADOOP_CONF_DIR` in `$SPARK_HOME/spark-env.sh` +to a location containing the configuration files. 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 d264ecbeaf7d8..b403292d9cb2e 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 @@ -375,9 +375,11 @@ object ClientBase { val classpathEntries = Option(conf.getStrings( YarnConfiguration.YARN_APPLICATION_CLASSPATH)).getOrElse( getDefaultYarnApplicationClasspath()) - for (c <- classpathEntries) { - YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, c.trim, - File.pathSeparator) + if (classpathEntries != null) { + for (c <- classpathEntries) { + YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, c.trim, + File.pathSeparator) + } } val mrClasspathEntries = Option(conf.getStrings( From 2de573877fbed20092f1b3af20b603b30ba9a940 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 22 Apr 2014 19:24:03 -0700 Subject: [PATCH 135/641] [Spark-1538] Fix SparkUI incorrectly hiding persisted RDDs **Bug**: After the following command `sc.parallelize(1 to 1000).persist.map(_ + 1).count()` is run, the the persisted RDD is missing from the storage tab of the SparkUI. **Cause**: The command creates two RDDs in one stage, a `ParallelCollectionRDD` and a `MappedRDD`. However, the existing StageInfo only keeps the RDDInfo of the last RDD associated with the stage (`MappedRDD`), and so all RDD information regarding the first RDD (`ParallelCollectionRDD`) is discarded. In this case, we persist the first RDD, but the StorageTab doesn't know about this RDD because it is not encoded in the StageInfo. **Fix**: Record information of all RDDs in StageInfo, instead of just the last RDD (i.e. `stage.rdd`). Since stage boundaries are marked by shuffle dependencies, the solution is to traverse the last RDD's dependency tree, visiting only ancestor RDDs related through a sequence of narrow dependencies. --- This PR also moves RDDInfo to its own file, includes a few style fixes, and adds a unit test for constructing StageInfos. Author: Andrew Or Closes #469 from andrewor14/storage-ui-fix and squashes the following commits: 07fc7f0 [Andrew Or] Add back comment that was accidentally removed (minor) 5d799fe [Andrew Or] Add comment to justify testing of getNarrowAncestors with cycles 9d0e2b8 [Andrew Or] Hide details of getNarrowAncestors from outsiders d2bac8a [Andrew Or] Deal with cycles in RDD dependency graph + add extensive tests 2acb177 [Andrew Or] Move getNarrowAncestors to RDD.scala bfe83f0 [Andrew Or] Backtrace RDD dependency tree to find all RDDs that belong to a Stage --- .../scala/org/apache/spark/TaskContext.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 2 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 25 +++ .../apache/spark/scheduler/JobLogger.scala | 2 +- .../apache/spark/scheduler/StageInfo.scala | 19 ++- .../org/apache/spark/storage/RDDInfo.scala | 55 +++++++ .../apache/spark/storage/StorageUtils.scala | 44 +----- .../apache/spark/ui/exec/ExecutorsPage.scala | 6 +- .../apache/spark/ui/storage/StorageTab.scala | 4 +- .../org/apache/spark/util/JsonProtocol.scala | 22 +-- .../org/apache/spark/CacheManagerSuite.scala | 6 +- .../org/apache/spark/PipedRDDSuite.scala | 2 +- .../scala/org/apache/spark/rdd/RDDSuite.scala | 148 +++++++++++++++++- .../spark/scheduler/SparkListenerSuite.scala | 54 ++++++- .../apache/spark/util/JsonProtocolSuite.scala | 8 +- 15 files changed, 318 insertions(+), 81 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/storage/RDDInfo.scala diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index dc5a19ecd738e..dc012cc381346 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -33,7 +33,7 @@ class TaskContext( val attemptId: Long, val runningLocally: Boolean = false, @volatile var interrupted: Boolean = false, - private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty() + private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty ) extends Serializable { @deprecated("use partitionId", "0.8.1") diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index e4f02a4be0b97..350fd74173f65 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -84,7 +84,7 @@ class TaskMetrics extends Serializable { } private[spark] object TaskMetrics { - def empty(): TaskMetrics = new TaskMetrics + def empty: TaskMetrics = new TaskMetrics } 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 5d2ed2b402cd8..596dcb84db7bf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import java.util.Random import scala.collection.Map +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.reflect.{classTag, ClassTag} @@ -229,6 +230,30 @@ abstract class RDD[T: ClassTag]( } } + /** + * Return the ancestors of the given RDD that are related to it only through a sequence of + * narrow dependencies. This traverses the given RDD's dependency tree using DFS, but maintains + * no ordering on the RDDs returned. + */ + private[spark] def getNarrowAncestors: Seq[RDD[_]] = { + val ancestors = new mutable.HashSet[RDD[_]] + + def visit(rdd: RDD[_]) { + val narrowDependencies = rdd.dependencies.filter(_.isInstanceOf[NarrowDependency[_]]) + val narrowParents = narrowDependencies.map(_.rdd) + val narrowParentsNotVisited = narrowParents.filterNot(ancestors.contains) + narrowParentsNotVisited.foreach { parent => + ancestors.add(parent) + visit(parent) + } + } + + visit(this) + + // In case there is a cycle, do not include the root itself + ancestors.filterNot(_ == this).toSeq + } + /** * Compute an RDD partition or read it from a checkpoint if the RDD is checkpointing. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 713aebfa3ce00..a1e21cad48b9b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -207,7 +207,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val taskInfo = taskEnd.taskInfo var taskStatus = "TASK_TYPE=%s".format(taskEnd.taskType) - val taskMetrics = if (taskEnd.taskMetrics != null) taskEnd.taskMetrics else TaskMetrics.empty() + val taskMetrics = if (taskEnd.taskMetrics != null) taskEnd.taskMetrics else TaskMetrics.empty taskEnd.reason match { case Success => taskStatus += " STATUS=SUCCESS" recordTaskMetrics(taskEnd.stageId, taskStatus, taskInfo, taskMetrics) diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 9f732f7191465..b42e231e11f91 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -25,7 +25,7 @@ import org.apache.spark.storage.RDDInfo * Stores information about a stage to pass from the scheduler to SparkListeners. */ @DeveloperApi -class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { +class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfos: Seq[RDDInfo]) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None /** Time when all tasks in the stage completed or when the stage was cancelled. */ @@ -41,12 +41,17 @@ class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddIn } } -private[spark] -object StageInfo { +private[spark] object StageInfo { + /** + * Construct a StageInfo from a Stage. + * + * Each Stage is associated with one or many RDDs, with the boundary of a Stage marked by + * shuffle dependencies. Therefore, all ancestor RDDs related to this Stage's RDD through a + * sequence of narrow dependencies should also be associated with this Stage. + */ def fromStage(stage: Stage): StageInfo = { - val rdd = stage.rdd - val rddName = Option(rdd.name).getOrElse(rdd.id.toString) - val rddInfo = new RDDInfo(rdd.id, rddName, rdd.partitions.size, rdd.getStorageLevel) - new StageInfo(stage.id, stage.name, stage.numTasks, rddInfo) + val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd) + val rddInfos = Seq(RDDInfo.fromRdd(stage.rdd)) ++ ancestorRddInfos + new StageInfo(stage.id, stage.name, stage.numTasks, rddInfos) } } diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala new file mode 100644 index 0000000000000..023fd6e4d8baa --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -0,0 +1,55 @@ +/* + * 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.storage + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +@DeveloperApi +class RDDInfo( + val id: Int, + val name: String, + val numPartitions: Int, + val storageLevel: StorageLevel) + extends Ordered[RDDInfo] { + + var numCachedPartitions = 0 + var memSize = 0L + var diskSize = 0L + var tachyonSize = 0L + + override def toString = { + import Utils.bytesToString + ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; " + + "TachyonSize: %s; DiskSize: %s").format( + name, id, storageLevel.toString, numCachedPartitions, numPartitions, + bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize)) + } + + override def compare(that: RDDInfo) = { + this.id - that.id + } +} + +private[spark] object RDDInfo { + def fromRdd(rdd: RDD[_]): RDDInfo = { + val rddName = Option(rdd.name).getOrElse(rdd.id.toString) + new RDDInfo(rdd.id, rddName, rdd.partitions.size, rdd.getStorageLevel) + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 7ed371326855d..1eddd1cdc483b 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -21,60 +21,30 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.SparkContext -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.util.Utils -private[spark] -class StorageStatus( +/** Storage information for each BlockManager. */ +private[spark] class StorageStatus( val blockManagerId: BlockManagerId, val maxMem: Long, val blocks: mutable.Map[BlockId, BlockStatus] = mutable.Map.empty) { - def memUsed() = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) + def memUsed = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) def memUsedByRDD(rddId: Int) = rddBlocks.filterKeys(_.rddId == rddId).values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) - def diskUsed() = blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + def diskUsed = blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) def diskUsedByRDD(rddId: Int) = rddBlocks.filterKeys(_.rddId == rddId).values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) - def memRemaining : Long = maxMem - memUsed() + def memRemaining: Long = maxMem - memUsed def rddBlocks = blocks.collect { case (rdd: RDDBlockId, status) => (rdd, status) } } -@DeveloperApi -private[spark] -class RDDInfo( - val id: Int, - val name: String, - val numPartitions: Int, - val storageLevel: StorageLevel) - extends Ordered[RDDInfo] { - - var numCachedPartitions = 0 - var memSize = 0L - var diskSize = 0L - var tachyonSize = 0L - - override def toString = { - import Utils.bytesToString - ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s;" + - "TachyonSize: %s; DiskSize: %s").format( - name, id, storageLevel.toString, numCachedPartitions, numPartitions, - bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize)) - } - - override def compare(that: RDDInfo) = { - this.id - that.id - } -} - -/* Helper methods for storage-related objects */ -private[spark] -object StorageUtils { +/** Helper methods for storage-related objects. */ +private[spark] object StorageUtils { /** * Returns basic information of all RDDs persisted in the given SparkContext. This does not diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index c1e69f6cdaffb..6cb43c02b8f08 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -32,7 +32,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = listener.storageStatusList val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) - val memUsed = storageStatusList.map(_.memUsed()).fold(0L)(_ + _) + val memUsed = storageStatusList.map(_.memUsed).fold(0L)(_ + _) val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_ + _) val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) val execInfoSorted = execInfo.sortBy(_.getOrElse("Executor ID", "")) @@ -106,9 +106,9 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { val execId = status.blockManagerId.executorId val hostPort = status.blockManagerId.hostPort val rddBlocks = status.blocks.size - val memUsed = status.memUsed() + val memUsed = status.memUsed val maxMem = status.maxMem - val diskUsed = status.diskUsed() + val diskUsed = status.diskUsed val activeTasks = listener.executorToTasksActive.getOrElse(execId, 0) val failedTasks = listener.executorToTasksFailed.getOrElse(execId, 0) val completedTasks = listener.executorToTasksComplete.getOrElse(execId, 0) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index c04ef0ae37748..07ec297841f20 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -66,8 +66,8 @@ private[ui] class StorageListener(storageStatusListener: StorageStatusListener) } override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { - val rddInfo = stageSubmitted.stageInfo.rddInfo - _rddInfoMap.getOrElseUpdate(rddInfo.id, rddInfo) + val rddInfos = stageSubmitted.stageInfo.rddInfos + rddInfos.foreach { info => _rddInfoMap.getOrElseUpdate(info.id, info) } } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 465835ea7fe29..9aed3e0985654 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -176,7 +176,7 @@ private[spark] object JsonProtocol { * -------------------------------------------------------------------- */ def stageInfoToJson(stageInfo: StageInfo): JValue = { - val rddInfo = rddInfoToJson(stageInfo.rddInfo) + val rddInfo = JArray(stageInfo.rddInfos.map(rddInfoToJson).toList) val submissionTime = stageInfo.submissionTime.map(JInt(_)).getOrElse(JNothing) val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing) val failureReason = stageInfo.failureReason.map(JString(_)).getOrElse(JNothing) @@ -208,7 +208,8 @@ private[spark] object JsonProtocol { taskMetrics.shuffleReadMetrics.map(shuffleReadMetricsToJson).getOrElse(JNothing) val shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics.map(shuffleWriteMetricsToJson).getOrElse(JNothing) - val updatedBlocks = taskMetrics.updatedBlocks.map { blocks => + val updatedBlocks = + taskMetrics.updatedBlocks.map { blocks => JArray(blocks.toList.map { case (id, status) => ("Block ID" -> id.toString) ~ ("Status" -> blockStatusToJson(status)) @@ -467,13 +468,13 @@ private[spark] object JsonProtocol { val stageId = (json \ "Stage ID").extract[Int] val stageName = (json \ "Stage Name").extract[String] val numTasks = (json \ "Number of Tasks").extract[Int] - val rddInfo = rddInfoFromJson(json \ "RDD Info") + val rddInfos = (json \ "RDD Info").extract[List[JValue]].map(rddInfoFromJson) val submissionTime = Utils.jsonOption(json \ "Submission Time").map(_.extract[Long]) val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]) val failureReason = Utils.jsonOption(json \ "Failure Reason").map(_.extract[String]) val emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] - val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfo) + val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfos) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime stageInfo.failureReason = failureReason @@ -518,13 +519,14 @@ private[spark] object JsonProtocol { Utils.jsonOption(json \ "Shuffle Read Metrics").map(shuffleReadMetricsFromJson) metrics.shuffleWriteMetrics = Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) - metrics.updatedBlocks = Utils.jsonOption(json \ "Updated Blocks").map { value => - value.extract[List[JValue]].map { block => - val id = BlockId((block \ "Block ID").extract[String]) - val status = blockStatusFromJson(block \ "Status") - (id, status) + metrics.updatedBlocks = + Utils.jsonOption(json \ "Updated Blocks").map { value => + value.extract[List[JValue]].map { block => + val id = BlockId((block \ "Block ID").extract[String]) + val status = blockStatusFromJson(block \ "Status") + (id, status) + } } - } metrics } diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index b86923f07f793..fd5b0906e6765 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -60,7 +60,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar whenExecuting(blockManager) { val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = TaskMetrics.empty()) + taskMetrics = TaskMetrics.empty) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } @@ -73,7 +73,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar whenExecuting(blockManager) { val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = TaskMetrics.empty()) + taskMetrics = TaskMetrics.empty) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(5, 6, 7)) } @@ -87,7 +87,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar whenExecuting(blockManager) { val context = new TaskContext(0, 0, 0, runningLocally = true, interrupted = false, - taskMetrics = TaskMetrics.empty()) + taskMetrics = TaskMetrics.empty) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index dfe057515efa0..0bb6a6b09c5b5 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -179,7 +179,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { val hadoopPart1 = generateFakeHadoopPartition() val pipedRdd = new PipedRDD(nums, "printenv " + varName) val tContext = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = TaskMetrics.empty()) + taskMetrics = TaskMetrics.empty) val rddIter = pipedRdd.compute(hadoopPart1, tContext) val arr = rddIter.toArray assert(arr(0) == "/some/path") 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 1901330d8b188..d7c90346d88a2 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -17,12 +17,10 @@ package org.apache.spark.rdd -import scala.collection.mutable.HashMap -import scala.collection.parallel.mutable +import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.reflect.ClassTag import org.scalatest.FunSuite -import org.scalatest.concurrent.Timeouts._ -import org.scalatest.time.{Millis, Span} import org.apache.spark._ import org.apache.spark.SparkContext._ @@ -153,7 +151,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { if (shouldFail) { throw new Exception("injected failure") } else { - return Array(1, 2, 3, 4).iterator + Array(1, 2, 3, 4).iterator } } }.cache() @@ -568,4 +566,144 @@ class RDDSuite extends FunSuite with SharedSparkContext { val ids = ranked.map(_._1).distinct().collect() assert(ids.length === n) } + + test("getNarrowAncestors") { + val rdd1 = sc.parallelize(1 to 100, 4) + val rdd2 = rdd1.filter(_ % 2 == 0).map(_ + 1) + val rdd3 = rdd2.map(_ - 1).filter(_ < 50).map(i => (i, i)) + val rdd4 = rdd3.reduceByKey(_ + _) + val rdd5 = rdd4.mapValues(_ + 1).mapValues(_ + 2).mapValues(_ + 3) + val ancestors1 = rdd1.getNarrowAncestors + val ancestors2 = rdd2.getNarrowAncestors + val ancestors3 = rdd3.getNarrowAncestors + val ancestors4 = rdd4.getNarrowAncestors + val ancestors5 = rdd5.getNarrowAncestors + + // Simple dependency tree with a single branch + assert(ancestors1.size === 0) + assert(ancestors2.size === 2) + assert(ancestors2.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 1) + assert(ancestors2.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors3.size === 5) + assert(ancestors3.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 1) + assert(ancestors3.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors3.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) + + // Any ancestors before the shuffle are not considered + assert(ancestors4.size === 1) + assert(ancestors4.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 1) + assert(ancestors5.size === 4) + assert(ancestors5.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 1) + assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 1) + assert(ancestors5.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 2) + } + + test("getNarrowAncestors with multiple parents") { + val rdd1 = sc.parallelize(1 to 100, 5) + val rdd2 = sc.parallelize(1 to 200, 10).map(_ + 1) + val rdd3 = sc.parallelize(1 to 300, 15).filter(_ > 50) + val rdd4 = rdd1.map(i => (i, i)) + val rdd5 = rdd2.map(i => (i, i)) + val rdd6 = sc.union(rdd1, rdd2) + val rdd7 = sc.union(rdd1, rdd2, rdd3) + val rdd8 = sc.union(rdd6, rdd7) + val rdd9 = rdd4.join(rdd5) + val ancestors6 = rdd6.getNarrowAncestors + val ancestors7 = rdd7.getNarrowAncestors + val ancestors8 = rdd8.getNarrowAncestors + val ancestors9 = rdd9.getNarrowAncestors + + // Simple dependency tree with multiple branches + assert(ancestors6.size === 3) + assert(ancestors6.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 2) + assert(ancestors6.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) + assert(ancestors7.size === 5) + assert(ancestors7.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 3) + assert(ancestors7.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) + assert(ancestors7.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + + // Dependency tree with duplicate nodes (e.g. rdd1 should not be reported twice) + assert(ancestors8.size === 7) + assert(ancestors8.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) + assert(ancestors8.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors8.count(_.isInstanceOf[UnionRDD[_]]) === 2) + assert(ancestors8.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 3) + assert(ancestors8.count(_ == rdd1) === 1) + assert(ancestors8.count(_ == rdd2) === 1) + assert(ancestors8.count(_ == rdd3) === 1) + + // Any ancestors before the shuffle are not considered + assert(ancestors9.size === 2) + assert(ancestors9.count(_.isInstanceOf[CoGroupedRDD[_]]) === 1) + assert(ancestors9.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 1) + } + + /** + * This tests for the pathological condition in which the RDD dependency graph is cyclical. + * + * Since RDD is part of the public API, applications may actually implement RDDs that allow + * such graphs to be constructed. In such cases, getNarrowAncestor should not simply hang. + */ + test("getNarrowAncestors with cycles") { + val rdd1 = new CyclicalDependencyRDD[Int] + val rdd2 = new CyclicalDependencyRDD[Int] + val rdd3 = new CyclicalDependencyRDD[Int] + val rdd4 = rdd3.map(_ + 1).filter(_ > 10).map(_ + 2).filter(_ % 5 > 1) + val rdd5 = rdd4.map(_ + 2).filter(_ > 20) + val rdd6 = sc.union(rdd1, rdd2, rdd3).map(_ + 4).union(rdd5).union(rdd4) + + // Simple cyclical dependency + rdd1.addDependency(new OneToOneDependency[Int](rdd2)) + rdd2.addDependency(new OneToOneDependency[Int](rdd1)) + val ancestors1 = rdd1.getNarrowAncestors + val ancestors2 = rdd2.getNarrowAncestors + assert(ancestors1.size === 1) + assert(ancestors1.count(_ == rdd2) === 1) + assert(ancestors1.count(_ == rdd1) === 0) + assert(ancestors2.size === 1) + assert(ancestors2.count(_ == rdd1) === 1) + assert(ancestors2.count(_ == rdd2) === 0) + + // Cycle involving a longer chain + rdd3.addDependency(new OneToOneDependency[Int](rdd4)) + val ancestors3 = rdd3.getNarrowAncestors + val ancestors4 = rdd4.getNarrowAncestors + assert(ancestors3.size === 4) + assert(ancestors3.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) + assert(ancestors3.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors3.count(_ == rdd3) === 0) + assert(ancestors4.size === 4) + assert(ancestors4.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) + assert(ancestors4.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors4.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 1) + assert(ancestors4.count(_ == rdd3) === 1) + assert(ancestors4.count(_ == rdd4) === 0) + + // Cycles that do not involve the root + val ancestors5 = rdd5.getNarrowAncestors + assert(ancestors5.size === 6) + assert(ancestors5.count(_.isInstanceOf[MappedRDD[_, _]]) === 3) + assert(ancestors5.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors5.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 1) + assert(ancestors4.count(_ == rdd3) === 1) + + // Complex cyclical dependency graph (combination of all of the above) + val ancestors6 = rdd6.getNarrowAncestors + assert(ancestors6.size === 12) + assert(ancestors6.count(_.isInstanceOf[UnionRDD[_]]) === 2) + assert(ancestors6.count(_.isInstanceOf[MappedRDD[_, _]]) === 4) + assert(ancestors6.count(_.isInstanceOf[FilteredRDD[_]]) === 3) + assert(ancestors6.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 3) + } + + /** A contrived RDD that allows the manual addition of dependencies after creation. */ + private class CyclicalDependencyRDD[T: ClassTag] extends RDD[T](sc, Nil) { + private val mutableDependencies: ArrayBuffer[Dependency[_]] = ArrayBuffer.empty + override def compute(p: Partition, c: TaskContext): Iterator[T] = Iterator.empty + override def getPartitions: Array[Partition] = Array.empty + override def getDependencies: Seq[Dependency[_]] = mutableDependencies + def addDependency(dep: Dependency[_]) { + mutableDependencies += dep + } + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 36511a9e95474..ab139175e00eb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -133,20 +133,57 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc val rdd1 = sc.parallelize(1 to 100, 4) val rdd2 = rdd1.map(_.toString) rdd2.setName("Target RDD") - rdd2.count + rdd2.count() assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {1} val (stageInfo, taskInfoMetrics) = listener.stageInfos.head - stageInfo.rddInfo.name should be {"Target RDD"} + stageInfo.rddInfos.size should be {2} + stageInfo.rddInfos.forall(_.numPartitions == 4) should be {true} + stageInfo.rddInfos.exists(_.name == "Target RDD") should be {true} stageInfo.numTasks should be {4} - stageInfo.rddInfo.numPartitions should be {4} stageInfo.submissionTime should be ('defined) stageInfo.completionTime should be ('defined) taskInfoMetrics.length should be {4} } + test("basic creation of StageInfo with shuffle") { + val listener = new SaveStageAndTaskInfo + sc.addSparkListener(listener) + val rdd1 = sc.parallelize(1 to 100, 4) + val rdd2 = rdd1.filter(_ % 2 == 0).map(i => (i, i)) + val rdd3 = rdd2.reduceByKey(_ + _) + rdd1.setName("Un") + rdd2.setName("Deux") + rdd3.setName("Trois") + + rdd1.count() + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + listener.stageInfos.size should be {1} + val stageInfo1 = listener.stageInfos.keys.find(_.stageId == 0).get + stageInfo1.rddInfos.size should be {1} // ParallelCollectionRDD + stageInfo1.rddInfos.forall(_.numPartitions == 4) should be {true} + stageInfo1.rddInfos.exists(_.name == "Un") should be {true} + listener.stageInfos.clear() + + rdd2.count() + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + listener.stageInfos.size should be {1} + val stageInfo2 = listener.stageInfos.keys.find(_.stageId == 1).get + stageInfo2.rddInfos.size should be {3} // ParallelCollectionRDD, FilteredRDD, MappedRDD + stageInfo2.rddInfos.forall(_.numPartitions == 4) should be {true} + stageInfo2.rddInfos.exists(_.name == "Deux") should be {true} + listener.stageInfos.clear() + + rdd3.count() + listener.stageInfos.size should be {2} // Shuffle map stage + result stage + val stageInfo3 = listener.stageInfos.keys.find(_.stageId == 2).get + stageInfo3.rddInfos.size should be {2} // ShuffledRDD, MapPartitionsRDD + stageInfo3.rddInfos.forall(_.numPartitions == 4) should be {true} + stageInfo3.rddInfos.exists(_.name == "Trois") should be {true} + } + test("StageInfo with fewer tasks than partitions") { val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) @@ -159,7 +196,8 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc listener.stageInfos.size should be {1} val (stageInfo, _) = listener.stageInfos.head stageInfo.numTasks should be {2} - stageInfo.rddInfo.numPartitions should be {4} + stageInfo.rddInfos.size should be {2} + stageInfo.rddInfos.forall(_.numPartitions == 4) should be {true} } test("local metrics") { @@ -167,7 +205,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc sc.addSparkListener(listener) sc.addSparkListener(new StatsReportListener) // just to make sure some of the tasks take a noticeable amount of time - val w = {i:Int => + val w = { i: Int => if (i == 0) Thread.sleep(100) i @@ -199,7 +237,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc checkNonZeroAvg( taskInfoMetrics.map(_._2.executorDeserializeTime), stageInfo + " executorDeserializeTime") - if (stageInfo.rddInfo.name == d4.name) { + if (stageInfo.rddInfos.exists(_.name == d4.name)) { checkNonZeroAvg( taskInfoMetrics.map(_._2.shuffleReadMetrics.get.fetchWaitTime), stageInfo + " fetchWaitTime") @@ -207,11 +245,11 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc taskInfoMetrics.foreach { case (taskInfo, taskMetrics) => taskMetrics.resultSize should be > (0l) - if (stageInfo.rddInfo.name == d2.name || stageInfo.rddInfo.name == d3.name) { + if (stageInfo.rddInfos.exists(info => info.name == d2.name || info.name == d3.name)) { taskMetrics.shuffleWriteMetrics should be ('defined) taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0l) } - if (stageInfo.rddInfo.name == d4.name) { + if (stageInfo.rddInfos.exists(_.name == d4.name)) { taskMetrics.shuffleReadMetrics should be ('defined) val sm = taskMetrics.shuffleReadMetrics.get sm.totalBlocksFetched should be > (0) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 16470bb7bf60d..3031015256ec9 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -231,7 +231,10 @@ class JsonProtocolSuite extends FunSuite { assert(info1.submissionTime === info2.submissionTime) assert(info1.completionTime === info2.completionTime) assert(info1.emittedTaskSizeWarning === info2.emittedTaskSizeWarning) - assertEquals(info1.rddInfo, info2.rddInfo) + assert(info1.rddInfos.size === info2.rddInfos.size) + (0 until info1.rddInfos.size).foreach { i => + assertEquals(info1.rddInfos(i), info2.rddInfos(i)) + } } private def assertEquals(info1: RDDInfo, info2: RDDInfo) { @@ -434,7 +437,8 @@ class JsonProtocolSuite extends FunSuite { } private def makeStageInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { - new StageInfo(a, "greetings", b, makeRddInfo(a, b, c, d, e)) + val rddInfos = (1 to a % 5).map { i => makeRddInfo(a % i, b % i, c % i, d % i, e % i) } + new StageInfo(a, "greetings", b, rddInfos) } private def makeTaskInfo(a: Long, b: Int, c: Long) = { From f3d19a9f1a4e38ff9fb5bf78e04ed5d321219bf6 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 22 Apr 2014 19:35:13 -0700 Subject: [PATCH 136/641] [streaming][SPARK-1578] Removed requirement for TTL in StreamingContext. Since shuffles and RDDs that are out of context are automatically cleaned by Spark core (using ContextCleaner) there is no need for setting the cleaner TTL while creating a StreamingContext. Author: Tathagata Das Closes #491 from tdas/ttl-fix and squashes the following commits: cf01dc7 [Tathagata Das] Removed requirement for TTL in StreamingContext. --- .../spark/streaming/StreamingContext.scala | 15 +------ .../spark/streaming/InputStreamsSuite.scala | 2 +- .../streaming/StreamingContextSuite.scala | 45 +++++-------------- .../spark/streaming/TestSuiteBase.scala | 1 - 4 files changed, 14 insertions(+), 49 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 6d9dc87a707ac..9ba6e02229aaa 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -116,11 +116,6 @@ class StreamingContext private[streaming] ( } } - if (MetadataCleaner.getDelaySeconds(sc.conf) < 0) { - throw new SparkException("Spark Streaming cannot be used without setting spark.cleaner.ttl; " - + "set this property before creating a SparkContext (use SPARK_JAVA_OPTS for the shell)") - } - private[streaming] val conf = sc.conf private[streaming] val env = SparkEnv.get @@ -500,8 +495,6 @@ class StreamingContext private[streaming] ( object StreamingContext extends Logging { - private[streaming] val DEFAULT_CLEANER_TTL = 3600 - implicit def toPairDStreamFunctions[K: ClassTag, V: ClassTag](stream: DStream[(K,V)]) = { new PairDStreamFunctions[K, V](stream) } @@ -546,13 +539,7 @@ object StreamingContext extends Logging { def jarOfClass(cls: Class[_]): Option[String] = SparkContext.jarOfClass(cls) private[streaming] def createNewSparkContext(conf: SparkConf): SparkContext = { - // Set the default cleaner delay to an hour if not already set. - // This should be sufficient for even 1 second batch intervals. - if (MetadataCleaner.getDelaySeconds(conf) < 0) { - MetadataCleaner.setDelaySeconds(conf, DEFAULT_CLEANER_TTL) - } - val sc = new SparkContext(conf) - sc + new SparkContext(conf) } private[streaming] def createNewSparkContext( diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 46b7f63b6567e..3bad871b5c580 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -143,7 +143,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } - // TODO: This test makes assumptions about Thread.sleep() and is flaky + // TODO: This test works in IntelliJ but not through SBT ignore("actor input stream") { // Start the server val testServer = new TestServer() diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 6d14b1f785db3..3e2b25af84098 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -38,15 +38,10 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val batchDuration = Milliseconds(500) val sparkHome = "someDir" val envPair = "key" -> "value" - val ttl = StreamingContext.DEFAULT_CLEANER_TTL + 100 var sc: SparkContext = null var ssc: StreamingContext = null - before { - System.clearProperty("spark.cleaner.ttl") - } - after { if (ssc != null) { ssc.stop() @@ -62,67 +57,51 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc = new StreamingContext(master, appName, batchDuration) assert(ssc.sparkContext.conf.get("spark.master") === master) assert(ssc.sparkContext.conf.get("spark.app.name") === appName) - assert(MetadataCleaner.getDelaySeconds(ssc.sparkContext.conf) === - StreamingContext.DEFAULT_CLEANER_TTL) } test("from no conf + spark home") { ssc = new StreamingContext(master, appName, batchDuration, sparkHome, Nil) assert(ssc.conf.get("spark.home") === sparkHome) - assert(MetadataCleaner.getDelaySeconds(ssc.sparkContext.conf) === - StreamingContext.DEFAULT_CLEANER_TTL) } test("from no conf + spark home + env") { ssc = new StreamingContext(master, appName, batchDuration, sparkHome, Nil, Map(envPair)) assert(ssc.conf.getExecutorEnv.exists(_ == envPair)) - assert(MetadataCleaner.getDelaySeconds(ssc.sparkContext.conf) === - StreamingContext.DEFAULT_CLEANER_TTL) - } - - test("from conf without ttl set") { - val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - ssc = new StreamingContext(myConf, batchDuration) - assert(MetadataCleaner.getDelaySeconds(ssc.conf) === - StreamingContext.DEFAULT_CLEANER_TTL) } - test("from conf with ttl set") { + test("from conf with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", ttl.toString) + myConf.set("spark.cleaner.ttl", "10") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === ttl) + assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) } - test("from existing SparkContext without ttl set") { + test("from existing SparkContext") { sc = new SparkContext(master, appName) - val exception = intercept[SparkException] { - ssc = new StreamingContext(sc, batchDuration) - } - assert(exception.getMessage.contains("ttl")) + ssc = new StreamingContext(sc, batchDuration) } - test("from existing SparkContext with ttl set") { + test("from existing SparkContext with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", ttl.toString) + myConf.set("spark.cleaner.ttl", "10") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === ttl) + assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) } test("from checkpoint") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", ttl.toString) + myConf.set("spark.cleaner.ttl", "10") val ssc1 = new StreamingContext(myConf, batchDuration) addInputStream(ssc1).register ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) - assert(MetadataCleaner.getDelaySeconds(cp.sparkConf) === ttl) + assert(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1") === "10") ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(MetadataCleaner.getDelaySeconds(newCp.sparkConf) === ttl) + assert(newCp.sparkConf.getInt("spark.cleaner.ttl", -1) === 10) ssc = new StreamingContext(null, newCp, null) - assert(MetadataCleaner.getDelaySeconds(ssc.conf) === ttl) + assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) } test("start and stop state check") { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index aa2d5c2fc2454..4f63fd37822cb 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -137,7 +137,6 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { val conf = new SparkConf() .setMaster(master) .setAppName(framework) - .set("spark.cleaner.ttl", StreamingContext.DEFAULT_CLEANER_TTL.toString) // Default before function for any streaming test suite. Override this // if you want to add your stuff to "before" (i.e., don't call before { } ) From 662c860ebcec5565493a7dff4812e6b7a37b1d7d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 22 Apr 2014 19:38:27 -0700 Subject: [PATCH 137/641] [FIX: SPARK-1376] use --arg instead of --args in SparkSubmit to avoid warning messages Even if users use `--arg`, `SparkSubmit` still uses `--args` for child args internally, which triggers a warning message that may confuse users: ~~~ --args is deprecated. Use --arg instead. ~~~ @sryza Does it look good to you? Author: Xiangrui Meng Closes #485 from mengxr/submit-arg and squashes the following commits: 5e1b9fe [Xiangrui Meng] update test cebbeb7 [Xiangrui Meng] use --arg instead of --args in SparkSubmit to avoid warning messages --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- .../test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 1b1e0fce0e0df..a2efddbfe3f69 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -196,7 +196,7 @@ object SparkSubmit { childArgs ++= appArgs.childArgs } else if (clusterManager == YARN) { for (arg <- appArgs.childArgs) { - childArgs += ("--args", arg) + childArgs += ("--arg", arg) } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index f82d717719b69..657b44668d385 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -110,7 +110,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { childArgsStr should include ("--executor-memory 5g") childArgsStr should include ("--driver-memory 4g") childArgsStr should include ("--executor-cores 5") - childArgsStr should include ("--args arg1 --args arg2") + childArgsStr should include ("--arg arg1 --arg arg2") childArgsStr should include ("--queue thequeue") childArgsStr should include ("--files file1.txt,file2.txt") childArgsStr should include ("--archives archive1.txt,archive2.txt") From aa77f8a6a604efe0d02bc8412b3f1ba3903b7a57 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 22 Apr 2014 20:02:33 -0700 Subject: [PATCH 138/641] SPARK-1562 Fix visibility / annotation of Spark SQL APIs Author: Michael Armbrust Closes #489 from marmbrus/sqlDocFixes and squashes the following commits: acee4f3 [Michael Armbrust] Fix visibility / annotation of Spark SQL APIs --- project/SparkBuild.scala | 4 +-- .../org/apache/spark/sql/SchemaRDDLike.scala | 2 +- .../spark/sql/execution/Aggregate.scala | 3 ++ .../apache/spark/sql/execution/Exchange.scala | 7 +++- .../apache/spark/sql/execution/Generate.scala | 3 ++ .../spark/sql/execution/SparkPlan.scala | 13 +++++-- .../sql/execution/SparkSqlSerializer.scala | 8 ++--- .../spark/sql/execution/SparkStrategies.scala | 2 +- .../spark/sql/execution/basicOperators.scala | 34 ++++++++++++++++++- .../apache/spark/sql/execution/debug.scala | 4 +-- .../apache/spark/sql/execution/joins.scala | 18 ++++++++++ .../apache/spark/sql/execution/package.scala | 1 + .../spark/sql/parquet/ParquetTestData.scala | 2 +- .../org/apache/spark/SparkHadoopWriter.scala | 9 ++--- .../spark/sql/hive/HiveMetastoreCatalog.scala | 8 +++-- .../org/apache/spark/sql/hive/HiveQl.scala | 14 ++++---- .../spark/sql/hive/HiveStrategies.scala | 3 +- .../spark/sql/hive/ScriptTransformation.scala | 6 +++- .../org/apache/spark/sql/hive/TestHive.scala | 3 +- .../apache/spark/sql/hive/hiveOperators.scala | 13 +++++-- .../org/apache/spark/sql/hive/hiveUdfs.scala | 18 +++++----- .../spark/sql/hive/CachedTableSuite.scala | 1 + .../sql/hive/InsertIntoHiveTableSuite.scala | 5 ++- .../sql/hive/api/java/JavaHiveSuite.scala | 2 +- .../execution/BigDataBenchmarkSuite.scala | 2 +- .../hive/execution/ConcurrentHiveSuite.scala | 2 +- .../hive/execution/HiveComparisonTest.scala | 2 +- .../execution/HiveCompatibilitySuite.scala | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 2 +- .../hive/execution/HiveResolutionSuite.scala | 4 +-- .../sql/hive/execution/PruningSuite.scala | 2 +- .../spark/sql/parquet/HiveParquetSuite.scala | 3 +- 32 files changed, 143 insertions(+), 59 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 1ca4e0dc8add1..852358501af56 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -367,7 +367,7 @@ object SparkBuild extends Build { publish := {}, unidocProjectFilter in (ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(repl, examples, tools, yarn, yarnAlpha), + inAnyProject -- inProjects(repl, examples, tools, catalyst, yarn, yarnAlpha), unidocProjectFilter in (JavaUnidoc, unidoc) := inAnyProject -- inProjects(repl, examples, bagel, graphx, catalyst, tools, yarn, yarnAlpha), @@ -457,7 +457,7 @@ object SparkBuild extends Build { def catalystSettings = sharedSettings ++ Seq( name := "catalyst", // The mechanics of rewriting expression ids to compare trees in some test cases makes - // assumptions about the the expression ids being contiguious. Running tests in parallel breaks + // assumptions about the the expression ids being contiguous. Running tests in parallel breaks // this non-deterministically. TODO: FIX THIS. parallelExecution in Test := false, libraryDependencies ++= Seq( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index a390ab6005dda..3a895e15a4508 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ /** * Contains functions that are shared between all SchemaRDD types (i.e., Scala, Java) */ -trait SchemaRDDLike { +private[sql] trait SchemaRDDLike { @transient val sqlContext: SQLContext @transient protected[spark] val logicalPlan: LogicalPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala index 3a4f071eebedf..36b3b956da96c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala @@ -19,12 +19,14 @@ package org.apache.spark.sql.execution import java.util.HashMap +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ /** + * :: DeveloperApi :: * Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each * group. * @@ -34,6 +36,7 @@ import org.apache.spark.sql.catalyst.plans.physical._ * @param aggregateExpressions expressions that are computed for each group. * @param child the input data source. */ +@DeveloperApi case class Aggregate( partial: Boolean, groupingExpressions: Seq[Expression], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 070557e47c4c7..3b4acb72e87b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf} import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.sql.Row @@ -26,6 +27,10 @@ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.util.MutablePair +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode { override def outputPartitioning = newPartitioning @@ -81,7 +86,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una * [[catalyst.plans.physical.Distribution Distribution]] requirements for each operator by inserting * [[Exchange]] Operators where required. */ -object AddExchange extends Rule[SparkPlan] { +private[sql] object AddExchange extends Rule[SparkPlan] { // TODO: Determine the number of partitions. val numPartitions = 150 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala index cff4887936ae1..da1e08be59e23 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala @@ -17,9 +17,11 @@ package org.apache.spark.sql.execution +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions.{Generator, JoinedRow, Literal, Projection} /** + * :: DeveloperApi :: * Applies a [[catalyst.expressions.Generator Generator]] to a stream of input rows, combining the * output of each into a new stream of rows. This operation is similar to a `flatMap` in functional * programming with one important additional feature, which allows the input rows to be joined with @@ -29,6 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.{Generator, JoinedRow, Literal, * @param outer when true, each input row will be output at least once, even if the output of the * given `generator` is empty. `outer` has no effect when `join` is false. */ +@DeveloperApi case class Generate( generator: Generator, join: Boolean, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 5d89697db5f99..50124dd407447 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Logging, Row} import org.apache.spark.sql.catalyst.trees @@ -26,6 +27,10 @@ import org.apache.spark.sql.catalyst.plans.{QueryPlan, logical} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.columnar.InMemoryColumnarTableScan +/** + * :: DeveloperApi :: + */ +@DeveloperApi abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { self: Product => @@ -51,6 +56,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { } /** + * :: DeveloperApi :: * Allows already planned SparkQueries to be linked into logical query plans. * * Note that in general it is not valid to use this class to link multiple copies of the same @@ -59,6 +65,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { * replace the output attributes with new copies of themselves without breaking any attribute * linking. */ +@DeveloperApi case class SparkLogicalPlan(alreadyPlanned: SparkPlan) extends logical.LogicalPlan with MultiInstanceRelation { @@ -77,15 +84,15 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan) } } -trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] { +private[sql] trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] { self: Product => } -trait UnaryNode extends SparkPlan with trees.UnaryNode[SparkPlan] { +private[sql] trait UnaryNode extends SparkPlan with trees.UnaryNode[SparkPlan] { self: Product => override def outputPartitioning: Partitioning = child.outputPartitioning } -trait BinaryNode extends SparkPlan with trees.BinaryNode[SparkPlan] { +private[sql] trait BinaryNode extends SparkPlan with trees.BinaryNode[SparkPlan] { self: Product => } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index c30ae5bcc02d0..5067c14ddffeb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -27,7 +27,7 @@ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.MutablePair import org.apache.spark.util.Utils -class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { +private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { override def newKryo(): Kryo = { val kryo = new Kryo() kryo.setRegistrationRequired(false) @@ -50,7 +50,7 @@ class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { } } -object SparkSqlSerializer { +private[sql] object SparkSqlSerializer { // TODO (lian) Using KryoSerializer here is workaround, needs further investigation // Using SparkSqlSerializer here makes BasicQuerySuite to fail because of Kryo serialization // related error. @@ -68,7 +68,7 @@ object SparkSqlSerializer { } } -class BigDecimalSerializer extends Serializer[BigDecimal] { +private[sql] class BigDecimalSerializer extends Serializer[BigDecimal] { def write(kryo: Kryo, output: Output, bd: math.BigDecimal) { // TODO: There are probably more efficient representations than strings... output.writeString(bd.toString()) @@ -83,7 +83,7 @@ class BigDecimalSerializer extends Serializer[BigDecimal] { * Maps do not have a no arg constructor and so cannot be serialized by default. So, we serialize * them as `Array[(k,v)]`. */ -class MapSerializer extends Serializer[Map[_,_]] { +private[sql] class MapSerializer extends Serializer[Map[_,_]] { def write(kryo: Kryo, output: Output, map: Map[_,_]) { kryo.writeObject(output, map.flatMap(e => Seq(e._1, e._2)).toArray) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index fe8bd5a508820..500fde19719af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.parquet._ -abstract class SparkStrategies extends QueryPlanner[SparkPlan] { +private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SQLContext#SparkPlanner => object HashJoin extends Strategy { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index eedcc7dda02d7..e4cf2020a1a93 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution import scala.reflect.runtime.universe.TypeTag +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.{HashPartitioner, SparkConf, SparkContext} import org.apache.spark.rdd.{RDD, ShuffledRDD} import org.apache.spark.sql.catalyst.ScalaReflection @@ -27,6 +28,10 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{OrderedDistribution, UnspecifiedDistribution} import org.apache.spark.util.MutablePair +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode { override def output = projectList.map(_.toAttribute) @@ -36,6 +41,10 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends } } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { override def output = child.output @@ -44,6 +53,10 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { } } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: SparkPlan) extends UnaryNode { @@ -53,6 +66,10 @@ case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: override def execute() = child.execute().sample(withReplacement, fraction, seed) } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Union(children: Seq[SparkPlan])(@transient sc: SparkContext) extends SparkPlan { // TODO: attributes output by union should be distinct for nullability purposes override def output = children.head.output @@ -62,12 +79,14 @@ case class Union(children: Seq[SparkPlan])(@transient sc: SparkContext) extends } /** + * :: DeveloperApi :: * Take the first limit elements. Note that the implementation is different depending on whether * this is a terminal operator or not. If it is terminal and is invoked using executeCollect, * this operator uses Spark's take method on the Spark driver. If it is not terminal or is * invoked using execute, we first take the limit on each partition, and then repartition all the * data to a single partition to compute the global limit. */ +@DeveloperApi case class Limit(limit: Int, child: SparkPlan)(@transient sc: SparkContext) extends UnaryNode { // TODO: Implement a partition local limit, and use a strategy to generate the proper limit plan: // partition local limit -> exchange into one partition -> partition local limit again @@ -91,10 +110,12 @@ case class Limit(limit: Int, child: SparkPlan)(@transient sc: SparkContext) exte } /** + * :: DeveloperApi :: * Take the first limit elements as defined by the sortOrder. This is logically equivalent to * having a [[Limit]] operator after a [[Sort]] operator. This could have been named TopK, but * Spark's top operator does the opposite in ordering so we name it TakeOrdered to avoid confusion. */ +@DeveloperApi case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) (@transient sc: SparkContext) extends UnaryNode { override def otherCopyArgs = sc :: Nil @@ -111,7 +132,10 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) override def execute() = sc.makeRDD(executeCollect(), 1) } - +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Sort( sortOrder: Seq[SortOrder], global: Boolean, @@ -134,6 +158,10 @@ case class Sort( override def output = child.output } +/** + * :: DeveloperApi :: + */ +@DeveloperApi object ExistingRdd { def convertToCatalyst(a: Any): Any = a match { case s: Seq[Any] => s.map(convertToCatalyst) @@ -167,6 +195,10 @@ object ExistingRdd { } } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { override def execute() = rdd } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala index 40982f1fffbbf..a0d29100f505a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -object DebugQuery { +private[sql] object DebugQuery { def apply(plan: SparkPlan): SparkPlan = { val visited = new collection.mutable.HashSet[Long]() plan transform { @@ -28,7 +28,7 @@ object DebugQuery { } } -case class DebugNode(child: SparkPlan) extends UnaryNode { +private[sql] case class DebugNode(child: SparkPlan) extends UnaryNode { def references = Set.empty def output = child.output def execute() = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index c89dae9358bf7..31cc26962ad93 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -21,14 +21,24 @@ import scala.collection.mutable.{ArrayBuffer, BitSet} import org.apache.spark.SparkContext +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning} +@DeveloperApi sealed abstract class BuildSide + +@DeveloperApi case object BuildLeft extends BuildSide + +@DeveloperApi case object BuildRight extends BuildSide +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class HashJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], @@ -130,6 +140,10 @@ case class HashJoin( } } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { def output = left.output ++ right.output @@ -138,6 +152,10 @@ case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNod } } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class BroadcastNestedLoopJoin( streamed: SparkPlan, broadcast: SparkPlan, joinType: JoinType, condition: Option[Expression]) (@transient sc: SparkContext) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala index e4a2dec33231d..66237f8f1314b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql /** + * :: DeveloperApi :: * An execution engine for relational query plans that runs on top Spark and returns RDDs. * * Note that the operators in this package are created automatically by a query planner using a diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 728e3dd1dc02b..f37976f7313c1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -28,7 +28,7 @@ import parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.util.Utils -object ParquetTestData { +private[sql] object ParquetTestData { val testSchema = """message myrecord { diff --git a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 7219c030cb0f0..ab7862f4f9e06 100644 --- a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.sql.hive import java.io.IOException import java.text.NumberFormat @@ -28,12 +28,13 @@ import org.apache.hadoop.hive.ql.plan.FileSinkDesc import org.apache.hadoop.mapred._ import org.apache.hadoop.io.Writable +import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} + /** * Internal helper class that saves an RDD using a Hive OutputFormat. * It is based on [[SparkHadoopWriter]]. */ -protected[spark] -class SparkHiveHadoopWriter( +private[hive] class SparkHiveHadoopWriter( @transient jobConf: JobConf, fileSinkConf: FileSinkDesc) extends Logging @@ -179,7 +180,7 @@ class SparkHiveHadoopWriter( } } -object SparkHiveHadoopWriter { +private[hive] object SparkHiveHadoopWriter { def createPathFromString(path: String, conf: JobConf): Path = { if (path == null) { throw new IllegalArgumentException("Output path is null") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index ca75cecf7d885..6c907887db79e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -34,12 +34,13 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.hive.execution.{HiveTableScan, InsertIntoHiveTable} import org.apache.spark.sql.columnar.InMemoryColumnarTableScan /* Implicit conversions */ import scala.collection.JavaConversions._ -class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { +private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { import HiveMetastoreTypes._ val client = Hive.get(hive.hiveconf) @@ -171,7 +172,7 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { override def unregisterAllTables() = {} } -object HiveMetastoreTypes extends RegexParsers { +private[hive] object HiveMetastoreTypes extends RegexParsers { protected lazy val primitiveType: Parser[DataType] = "string" ^^^ StringType | "float" ^^^ FloatType | @@ -229,7 +230,8 @@ object HiveMetastoreTypes extends RegexParsers { } } -case class MetastoreRelation(databaseName: String, tableName: String, alias: Option[String]) +private[hive] case class MetastoreRelation + (databaseName: String, tableName: String, alias: Option[String]) (val table: TTable, val partitions: Seq[TPartition]) extends BaseRelation { // TODO: Can we use org.apache.hadoop.hive.ql.metadata.Table as the type of table and diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 4dac25b3f60e4..1777e96b6713d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -36,20 +36,20 @@ import scala.collection.JavaConversions._ * back for Hive to execute natively. Will be replaced with a native command that contains the * cmd string. */ -case object NativePlaceholder extends Command +private[hive] case object NativePlaceholder extends Command -case class DfsCommand(cmd: String) extends Command +private[hive] case class DfsCommand(cmd: String) extends Command -case class ShellCommand(cmd: String) extends Command +private[hive] case class ShellCommand(cmd: String) extends Command -case class SourceCommand(filePath: String) extends Command +private[hive] case class SourceCommand(filePath: String) extends Command -case class AddJar(jarPath: String) extends Command +private[hive] case class AddJar(jarPath: String) extends Command -case class AddFile(filePath: String) extends Command +private[hive] case class AddFile(filePath: String) extends Command /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ -object HiveQl { +private[hive] object HiveQl { protected val nativeCommands = Seq( "TOK_DESCFUNCTION", "TOK_DESCTABLE", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index ac817b21a152e..d9a6e0e88932e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -23,9 +23,10 @@ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution._ +import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.columnar.InMemoryColumnarTableScan -trait HiveStrategies { +private[hive] trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. self: SQLContext#SparkPlanner => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala index 2610100043659..610fa9cb843c0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala @@ -15,23 +15,27 @@ * limitations under the License. */ -package org.apache.spark.sql.hive +package org.apache.spark.sql.hive.execution import java.io.{BufferedReader, InputStreamReader} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution._ +import org.apache.spark.sql.hive.HiveContext /* Implicit conversions */ import scala.collection.JavaConversions._ /** + * :: DeveloperApi :: * Transforms the input by forking and running the specified script. * * @param input the set of expression that should be passed to the script. * @param script the command that should be executed. * @param output the attributes that are produced by the script. */ +@DeveloperApi case class ScriptTransformation( input: Seq[Expression], script: String, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index b1a26fdabbb85..74110ee27b76c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.hive +package org.apache.spark.sql.hive.test import java.io.File import java.util.{Set => JavaSet} @@ -34,6 +34,7 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, NativeCommand} import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.hive._ /* Implicit conversions */ import scala.collection.JavaConversions._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala index 821fb22112f87..96faebc5a8687 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.hive +package org.apache.spark.sql.hive.execution import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} import org.apache.hadoop.hive.metastore.MetaStoreUtils @@ -30,23 +30,26 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharOb import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred._ +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.{BooleanType, DataType} import org.apache.spark.sql.execution._ -import org.apache.spark.{SparkHiveHadoopWriter, TaskContext, SparkException} +import org.apache.spark.sql.hive._ +import org.apache.spark.{TaskContext, SparkException} /* Implicits */ import scala.collection.JavaConversions._ /** + * :: DeveloperApi :: * The Hive table scan operator. Column and partition pruning are both handled. * - * @constructor * @param attributes Attributes to be fetched from the Hive table. * @param relation The Hive table be be scanned. * @param partitionPruningPred An optional partition pruning predicate for partitioned table. */ +@DeveloperApi case class HiveTableScan( attributes: Seq[Attribute], relation: MetastoreRelation, @@ -160,6 +163,10 @@ case class HiveTableScan( def output = attributes } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class InsertIntoHiveTable( table: MetastoreRelation, partition: Map[String, Option[String]], diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 55a4363af6c76..a09270eb7b134 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.catalyst.types._ /* Implicit conversions */ import scala.collection.JavaConversions._ -object HiveFunctionRegistry +private[hive] object HiveFunctionRegistry extends analysis.FunctionRegistry with HiveFunctionFactory with HiveInspectors { def lookupFunction(name: String, children: Seq[Expression]): Expression = { @@ -99,7 +99,7 @@ object HiveFunctionRegistry } } -trait HiveFunctionFactory { +private[hive] trait HiveFunctionFactory { def getFunctionInfo(name: String) = FunctionRegistry.getFunctionInfo(name) def getFunctionClass(name: String) = getFunctionInfo(name).getFunctionClass def createFunction[UDFType](name: String) = @@ -130,7 +130,7 @@ trait HiveFunctionFactory { } } -abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory { +private[hive] abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory { self: Product => type UDFType @@ -148,7 +148,7 @@ abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory override def toString = s"$nodeName#${functionInfo.getDisplayName}(${children.mkString(",")})" } -case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUdf { +private[hive] case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUdf { import org.apache.spark.sql.hive.HiveFunctionRegistry._ type UDFType = UDF @@ -201,7 +201,7 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd } } -case class HiveGenericUdf(name: String, children: Seq[Expression]) +private[hive] case class HiveGenericUdf(name: String, children: Seq[Expression]) extends HiveUdf with HiveInspectors { import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ @@ -228,7 +228,7 @@ case class HiveGenericUdf(name: String, children: Seq[Expression]) } } -trait HiveInspectors { +private[hive] trait HiveInspectors { def unwrapData(data: Any, oi: ObjectInspector): Any = oi match { case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) @@ -329,7 +329,7 @@ trait HiveInspectors { } } -case class HiveGenericUdaf( +private[hive] case class HiveGenericUdaf( name: String, children: Seq[Expression]) extends AggregateExpression with HiveInspectors @@ -371,7 +371,7 @@ case class HiveGenericUdaf( * Operators that require maintaining state in between input rows should instead be implemented as * user defined aggregations, which have clean semantics even in a partitioned execution. */ -case class HiveGenericUdtf( +private[hive] case class HiveGenericUdtf( name: String, aliasNames: Seq[String], children: Seq[Expression]) @@ -438,7 +438,7 @@ case class HiveGenericUdtf( override def toString = s"$nodeName#$name(${children.mkString(",")})" } -case class HiveUdafFunction( +private[hive] case class HiveUdafFunction( functionName: String, exprs: Seq[Expression], base: AggregateExpression) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 79ec1f1cde019..f9a162ef4e3c0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive import org.apache.spark.sql.execution.SparkLogicalPlan import org.apache.spark.sql.columnar.InMemoryColumnarTableScan import org.apache.spark.sql.hive.execution.HiveComparisonTest +import org.apache.spark.sql.hive.test.TestHive class CachedTableSuite extends HiveComparisonTest { TestHive.loadTestTable("src") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index ad29e06905c1b..833f3502154f3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -17,12 +17,11 @@ package org.apache.spark.sql.hive -import java.io.File - import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.hive.test.TestHive /* Implicits */ -import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.hive.test.TestHive._ case class TestData(key: Int, value: String) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala index 8137f99b227f4..9c5d7c81f7c09 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala @@ -21,7 +21,7 @@ import org.scalatest.FunSuite import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.test.TestSQLContext -import org.apache.spark.sql.hive.TestHive +import org.apache.spark.sql.hive.test.TestHive // Implicits import scala.collection.JavaConversions._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala index 8488f23abd0fd..9b9a823b6e95b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.execution import java.io.File -import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.hive.test.TestHive._ /** * A set of test cases based on the big-data-benchmark. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala index ac87f2cb10d12..23ece7e7cf6e9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.hive.TestHiveContext +import org.apache.spark.sql.hive.test.TestHiveContext import org.scalatest.{BeforeAndAfterAll, FunSuite} class ConcurrentHiveSuite extends FunSuite with BeforeAndAfterAll { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 6c91f40d0f925..ea17e6e93b41c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{ExplainCommand, NativeComman import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.Sort import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} -import org.apache.spark.sql.hive.TestHive +import org.apache.spark.sql.hive.test.TestHive /** * Allows the creations of tests that execute the same query against both hive diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index c3cfa3d25a5c2..dfe88b960b639 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.execution import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.hive.TestHive +import org.apache.spark.sql.hive.test.TestHive /** * Runs the test cases that are included in the hive distribution. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index a09667ac84b01..d224d2ee600ec 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.hive.test.TestHive._ /** * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index 8883e5b16d4da..67594b57d3dfa 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.hive.TestHive -import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ case class Data(a: Int, B: Int, n: Nested) case class Nested(a: Int, B: Int) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index d9ccb93e23923..25eca397462f7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.hive.{HiveTableScan, TestHive} +import org.apache.spark.sql.hive.test.TestHive /* Implicit conversions */ import scala.collection.JavaConversions._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index 843c681e0dc48..91ad59d7f82c0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -22,11 +22,10 @@ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row} import org.apache.spark.sql.catalyst.types.{DataType, StringType, IntegerType} import org.apache.spark.sql.{parquet, SchemaRDD} -import org.apache.spark.sql.hive.TestHive import org.apache.spark.util.Utils // Implicits -import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.hive.test.TestHive._ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { From 8e950813334dd8545357f59f4ed77c39ee4180b3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 22 Apr 2014 21:56:15 -0700 Subject: [PATCH 139/641] SPARK-1494 Don't initialize classes loaded by MIMA excludes. [WIP] Just seeing how Jenkins likes this... Author: Michael Armbrust Closes #494 from marmbrus/mima and squashes the following commits: 6eec616 [Michael Armbrust] Force hive tests to run. acaf682 [Michael Armbrust] Don't initialize loaded classes. --- .../scala/org/apache/spark/tools/GenerateMIMAIgnore.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala index 3fb85e1ff73a8..d9978b02a9f3e 100644 --- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala +++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala @@ -41,7 +41,7 @@ object GenerateMIMAIgnore { private def classesPrivateWithin(packageName: String): Set[String] = { - val classes = getClasses(packageName, classLoader) + val classes = getClasses(packageName) val privateClasses = mutable.HashSet[String]() def isPackagePrivate(className: String) = { @@ -108,8 +108,7 @@ object GenerateMIMAIgnore { * Scans all classes accessible from the context class loader which belong to the given package * and subpackages both from directories and jars present on the classpath. */ - private def getClasses(packageName: String, - classLoader: ClassLoader = Thread.currentThread().getContextClassLoader): Set[String] = { + private def getClasses(packageName: String): Set[String] = { val path = packageName.replace('.', '/') val resources = classLoader.getResources(path) @@ -128,7 +127,7 @@ object GenerateMIMAIgnore { val jar = new JarFile(new File(jarPath)) val enums = jar.entries().map(_.getName).filter(_.startsWith(packageName)) val classes = for (entry <- enums if entry.endsWith(".class")) - yield Class.forName(entry.replace('/', '.').stripSuffix(".class")) + yield Class.forName(entry.replace('/', '.').stripSuffix(".class"), false, classLoader) classes } } From 39f85e0322cfecefbc30e7d5a30356cfab1e9640 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 22 Apr 2014 22:19:32 -0700 Subject: [PATCH 140/641] [SQL] SPARK-1571 Mistake in java example code Author: Michael Armbrust Closes #496 from marmbrus/javaBeanBug and squashes the following commits: 644fedd [Michael Armbrust] Bean methods must be public. --- docs/sql-programming-guide.md | 8 ++++---- .../java/org/apache/spark/examples/sql/JavaSparkSQL.java | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index e25379bd7614d..3ca6cc95bd2db 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -143,19 +143,19 @@ public static class Person implements Serializable { private String name; private int age; - String getName() { + public String getName() { return name; } - void setName(String name) { + public void setName(String name) { this.name = name; } - int getAge() { + public int getAge() { return age; } - void setAge(int age) { + public void setAge(int age) { this.age = age; } } diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index e8e63d2745692..b5b438e9753a7 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -34,19 +34,19 @@ public static class Person implements Serializable { private String name; private int age; - String getName() { + public String getName() { return name; } - void setName(String name) { + public void setName(String name) { this.name = name; } - int getAge() { + public int getAge() { return age; } - void setAge(int age) { + public void setAge(int age) { this.age = age; } } From cd4ed293262e2349794c13467d1737974385c019 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 23 Apr 2014 10:19:32 -0700 Subject: [PATCH 141/641] SPARK-1119 and other build improvements 1. Makes assembly and examples jar naming consistent in maven/sbt. 2. Updates make-distribution.sh to use Maven and fixes some bugs. 3. Updates the create-release script to call make-distribution script. Author: Patrick Wendell Closes #502 from pwendell/make-distribution and squashes the following commits: 1a97f0d [Patrick Wendell] SPARK-1119 and other build improvements --- assembly/pom.xml | 2 +- bin/compute-classpath.sh | 4 +- bin/run-example | 11 +++-- dev/create-release/create-release.sh | 18 +++---- examples/pom.xml | 2 +- make-distribution.sh | 70 +++++++++++++++++++--------- project/SparkBuild.scala | 4 +- 7 files changed, 70 insertions(+), 41 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 923bf47f7076a..bdb38806492a6 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -33,7 +33,7 @@ scala-${scala.binary.version} - ${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar + spark-assembly-${project.version}-hadoop${hadoop.version}.jar ${project.build.directory}/${spark.jar.dir}/${spark.jar.basename} spark /usr/share/spark diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 2a2bb376fd71f..3a59f599fd7d2 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -50,9 +50,9 @@ if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then else # Else use spark-assembly jar from either RELEASE or assembly directory if [ -f "$FWDIR/RELEASE" ]; then - ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark*-assembly*.jar` + ASSEMBLY_JAR=`ls "$FWDIR"/lib/spark-assembly*hadoop*.jar` else - ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark*-assembly*hadoop*.jar` + ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*.jar` fi CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" fi diff --git a/bin/run-example b/bin/run-example index b2999198a8d41..d8a94f2e31e07 100755 --- a/bin/run-example +++ b/bin/run-example @@ -40,12 +40,15 @@ fi # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack # to avoid the -sources and -doc packages that are built by publish-local. EXAMPLES_DIR="$FWDIR"/examples -SPARK_EXAMPLES_JAR="" -if [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar` + +if [ -f "$FWDIR/RELEASE" ]; then + export SPARK_EXAMPLES_JAR=`ls "$FWDIR"/lib/spark-examples-*hadoop*.jar` +elif [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar ]; then + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar` fi + if [[ -z $SPARK_EXAMPLES_JAR ]]; then - echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2 + echo "Failed to find Spark examples assembly in $FWDIR/lib or $FWDIR/examples/target" >&2 echo "You need to build Spark with sbt/sbt assembly before running this program" >&2 exit 1 fi diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index fb9d9f9e072a6..ad38c8d53eb2d 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -83,15 +83,15 @@ rm -rf spark-$RELEASE_VERSION make_binary_release() { NAME=$1 - MAVEN_FLAGS=$2 - + FLAGS=$2 cp -r spark spark-$RELEASE_VERSION-bin-$NAME + cd spark-$RELEASE_VERSION-bin-$NAME - export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" - mvn $MAVEN_FLAGS -DskipTests clean package - find . -name test-classes -type d | xargs rm -rf - find . -name classes -type d | xargs rm -rf + ./make-distribution.sh $FLAGS --name $NAME --tgz cd .. + cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . + rm -rf spark-$RELEASE_VERSION-bin-$NAME + tar cvzf spark-$RELEASE_VERSION-bin-$NAME.tgz spark-$RELEASE_VERSION-bin-$NAME echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour \ --output spark-$RELEASE_VERSION-bin-$NAME.tgz.asc \ @@ -105,9 +105,9 @@ make_binary_release() { rm -rf spark-$RELEASE_VERSION-bin-$NAME } -make_binary_release "hadoop1" "-Dhadoop.version=1.0.4" -make_binary_release "cdh4" "-Dhadoop.version=2.0.0-mr1-cdh4.2.0" -make_binary_release "hadoop2" "-Pyarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0" +make_binary_release "hadoop1" "--hadoop 1.0.4" +make_binary_release "cdh4" "--hadoop 2.0.0-mr1-cdh4.2.0" +make_binary_release "hadoop2" "--with-yarn --hadoop 2.2.0" # Copy data echo "Copying release tarballs" diff --git a/examples/pom.xml b/examples/pom.xml index 0b6212b5d1549..704d6df7c57f9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -187,7 +187,7 @@ maven-shade-plugin false - ${project.build.directory}/scala-${scala.binary.version}/${project.artifactId}-assembly-${project.version}.jar + ${project.build.directory}/scala-${scala.binary.version}/spark-examples-${project.version}-hadoop${hadoop.version}.jar *:* diff --git a/make-distribution.sh b/make-distribution.sh index 5c780fcbda863..83dfc745857ca 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -28,6 +28,8 @@ # --tgz: Additionally creates spark-$VERSION-bin.tar.gz # --hadoop VERSION: Builds against specified version of Hadoop. # --with-yarn: Enables support for Hadoop YARN. +# --with-hive: Enable support for reading Hive tables. +# --name: A moniker for the release target. Defaults to the Hadoop verison. # # Recommended deploy/testing procedure (standalone mode): # 1) Rsync / deploy the dist/ dir to one host @@ -41,25 +43,20 @@ FWDIR="$(cd `dirname $0`; pwd)" DISTDIR="$FWDIR/dist" -# Get version from SBT -export TERM=dumb # Prevents color codes in SBT output - -VERSIONSTRING=$($FWDIR/sbt/sbt "show version") - +VERSION=$(mvn help:evaluate -Dexpression=project.version |grep -v "INFO") if [ $? == -1 ] ;then - echo -e "You need sbt installed and available on your path." - echo -e "Download sbt from http://www.scala-sbt.org/" + echo -e "You need Maven installed to build Spark." + echo -e "Download Maven from https://maven.apache.org." exit -1; fi -VERSION=$(echo "${VERSIONSTRING}" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z0-9.-]*\).*/\1/') -echo "Version is ${VERSION}" - # Initialize defaults SPARK_HADOOP_VERSION=1.0.4 SPARK_YARN=false +SPARK_HIVE=false SPARK_TACHYON=false MAKE_TGZ=false +NAME=none # Parse arguments while (( "$#" )); do @@ -71,23 +68,37 @@ while (( "$#" )); do --with-yarn) SPARK_YARN=true ;; + --with-hive) + SPARK_HIVE=true + ;; --with-tachyon) SPARK_TACHYON=true ;; --tgz) MAKE_TGZ=true ;; + --name) + NAME="$2" + shift + ;; esac shift done +if [ "$NAME" == "none" ]; then + NAME=$SPARK_HADOOP_VERSION +fi + +echo "Spark version is $VERSION" + if [ "$MAKE_TGZ" == "true" ]; then - echo "Making spark-$VERSION-hadoop_$SPARK_HADOOP_VERSION-bin.tar.gz" + echo "Making spark-$VERSION-bin-$NAME.tgz" else - echo "Making distribution for Spark $VERSION in $DISTDIR..." + echo "Making distribution for Spark $VERSION in $DISTDIR..." fi echo "Hadoop version set to $SPARK_HADOOP_VERSION" +echo "Release name set to $NAME" if [ "$SPARK_YARN" == "true" ]; then echo "YARN enabled" else @@ -100,20 +111,32 @@ else echo "Tachyon Disabled" fi -# Build fat JAR -export SPARK_HADOOP_VERSION -export SPARK_YARN +# Build uber fat JAR cd $FWDIR -"sbt/sbt" "assembly/assembly" +export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" + +if [ "$SPARK_HIVE" == "true" ]; then + MAYBE_HIVE="-Phive" +else + MAYBE_HIVE="" +fi + +if [ "$SPARK_YARN" == "true" ]; then + mvn clean package -DskipTests -Pyarn -Dhadoop.version=$SPARK_HADOOP_VERSION \ + -Dyarn.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE +else + mvn clean package -DskipTests -Dhadoop.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE +fi # Make directories rm -rf "$DISTDIR" -mkdir -p "$DISTDIR/jars" +mkdir -p "$DISTDIR/lib" echo "Spark $VERSION built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE" # Copy jars -cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/jars/" +cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" +cp $FWDIR/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" # Copy other things mkdir "$DISTDIR"/conf @@ -135,16 +158,16 @@ if [ "$SPARK_TACHYON" == "true" ]; then wget "$TACHYON_URL" tar xf "tachyon-${TACHYON_VERSION}-bin.tar.gz" - cp "tachyon-${TACHYON_VERSION}/target/tachyon-${TACHYON_VERSION}-jar-with-dependencies.jar" "$DISTDIR/jars" + cp "tachyon-${TACHYON_VERSION}/target/tachyon-${TACHYON_VERSION}-jar-with-dependencies.jar" "$DISTDIR/lib" mkdir -p "$DISTDIR/tachyon/src/main/java/tachyon/web" cp -r "tachyon-${TACHYON_VERSION}"/{bin,conf,libexec} "$DISTDIR/tachyon" cp -r "tachyon-${TACHYON_VERSION}"/src/main/java/tachyon/web/resources "$DISTDIR/tachyon/src/main/java/tachyon/web" if [[ `uname -a` == Darwin* ]]; then # need to run sed differently on osx - nl=$'\n'; sed -i "" -e "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\\$nl export TACHYON_JAR=\$TACHYON_HOME/../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" + nl=$'\n'; sed -i "" -e "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\\$nl export TACHYON_JAR=\$TACHYON_HOME/../lib/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" else - sed -i "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\n export TACHYON_JAR=\$TACHYON_HOME/../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" + sed -i "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\n export TACHYON_JAR=\$TACHYON_HOME/../lib/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" fi popd > /dev/null @@ -152,8 +175,9 @@ if [ "$SPARK_TACHYON" == "true" ]; then fi if [ "$MAKE_TGZ" == "true" ]; then - TARDIR="$FWDIR/spark-$VERSION" + TARDIR_NAME=spark-$VERSION-bin-$NAME + TARDIR="$FWDIR/$TARDIR_NAME" cp -r "$DISTDIR" "$TARDIR" - tar -zcf "spark-$VERSION-hadoop_$SPARK_HADOOP_VERSION-bin.tar.gz" -C "$FWDIR" "spark-$VERSION" + tar czf "spark-$VERSION-bin-$NAME.tgz" -C "$FWDIR" "$TARDIR_NAME" rm -rf "$TARDIR" fi diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 852358501af56..b8af2bbd2ef6a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -411,7 +411,9 @@ object SparkBuild extends Build { ) def examplesSettings = sharedSettings ++ Seq( - name := "spark-examples", + name := "spark-examples", + jarName in assembly <<= version map { + v => "spark-examples-" + v + "-hadoop" + hadoopVersion + ".jar" }, libraryDependencies ++= Seq( "com.twitter" %% "algebird-core" % "0.1.11", "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging), From a66460661303bd652b5c5d32860216cc3ceb7f98 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 23 Apr 2014 14:12:20 -0700 Subject: [PATCH 142/641] SPARK-1583: Fix a bug that using java.util.HashMap by mistake JIRA: https://issues.apache.org/jira/browse/SPARK-1583 Does anyone know why using `java.util.HashMap` rather than `mutable.HashMap`? Some methods of `java.util.HashMap` are not generics and compiler can not help us find similar problems. Author: zsxwing Closes #500 from zsxwing/SPARK-1583 and squashes the following commits: 7bfd74d [zsxwing] SPARK-1583: Fix a bug that using java.util.HashMap by mistake --- .../org/apache/spark/storage/BlockManagerMasterActor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index c57b6e8391b13..63fa5d3eb6541 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -203,7 +203,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus val locations = blockLocations.get(blockId) locations -= blockManagerId if (locations.size == 0) { - blockLocations.remove(locations) + blockLocations.remove(blockId) } } listenerBus.post(SparkListenerBlockManagerRemoved(blockManagerId)) From a967b005c8937a3053e215c952d2172ee3dc300d Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Wed, 23 Apr 2014 14:46:30 -0700 Subject: [PATCH 143/641] SPARK-1572 Don't kill Executor if PythonRDD fails while computing parent Previously, the behavior was that if the parent RDD threw any exception other than IOException or FileNotFoundException (which is quite possible for Hadoop input sources), the entire Executor would crash, because the default thread a uncaught exception handler calls System.exit(). This patch avoids two related issues: 1. Always catch exceptions in this reader thread. 2. Don't mask readerException when Python throws an EOFError after worker.shutdownOutput() is called. Author: Aaron Davidson Closes #486 from aarondav/pyspark and squashes the following commits: fbb11e9 [Aaron Davidson] Make sure FileNotFoundExceptions are handled same as before b9acb3e [Aaron Davidson] SPARK-1572 Don't kill Executor if PythonRDD fails while computing parent --- .../apache/spark/api/python/PythonRDD.scala | 32 ++++++++++++------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index f9d86fed34d0f..8a843fbb0e66f 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 @@ -24,6 +24,7 @@ import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collectio import scala.collection.JavaConversions._ import scala.reflect.ClassTag +import scala.util.Try import net.razorvine.pickle.{Pickler, Unpickler} @@ -89,16 +90,22 @@ private[spark] class PythonRDD[T: ClassTag]( dataOut.flush() worker.shutdownOutput() } catch { + case e: java.io.FileNotFoundException => readerException = e - // Kill the Python worker process: - worker.shutdownOutput() + Try(worker.shutdownOutput()) // kill Python worker process + case e: IOException => // This can happen for legitimate reasons if the Python code stops returning data - // before we are done passing elements through, e.g., for take(). Just log a message - // to say it happened. - logInfo("stdin writer to Python finished early") - logDebug("stdin writer to Python finished early", e) + // before we are done passing elements through, e.g., for take(). Just log a message to + // say it happened (as it could also be hiding a real IOException from a data source). + logInfo("stdin writer to Python finished early (may not be an error)", e) + + case e: Exception => + // We must avoid throwing exceptions here, because the thread uncaught exception handler + // will kill the whole executor (see Executor). + readerException = e + Try(worker.shutdownOutput()) // kill Python worker process } } }.start() @@ -152,7 +159,7 @@ private[spark] class PythonRDD[T: ClassTag]( val exLength = stream.readInt() val obj = new Array[Byte](exLength) stream.readFully(obj) - throw new PythonException(new String(obj)) + throw new PythonException(new String(obj), readerException) case SpecialLengths.END_OF_DATA_SECTION => // We've finished the data section of the output, but we can still // read some accumulator updates: @@ -167,10 +174,13 @@ private[spark] class PythonRDD[T: ClassTag]( Array.empty[Byte] } } catch { - case eof: EOFException => { + case e: Exception if readerException != null => + logError("Python worker exited unexpectedly (crashed)", e) + logError("Python crash may have been caused by prior exception:", readerException) + throw readerException + + case eof: EOFException => throw new SparkException("Python worker exited unexpectedly (crashed)", eof) - } - case e: Throwable => throw e } } @@ -185,7 +195,7 @@ private[spark] class PythonRDD[T: ClassTag]( } /** Thrown for exceptions in user Python code. */ -private class PythonException(msg: String) extends Exception(msg) +private class PythonException(msg: String, cause: Exception) extends RuntimeException(msg, cause) /** * Form an RDD[(Array[Byte], Array[Byte])] from key-value pairs returned from Python. From dd1b7a61d9193c93ab95ab550622259f4bc26f53 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 23 Apr 2014 14:47:38 -0700 Subject: [PATCH 144/641] Honor default fs name when initializing event logger. This is related to SPARK-1459 / PR #375. Without this fix, FileLogger.createLogDir() may try to create the log dir on HDFS, while createWriter() will try to open the log file on the local file system, leading to interesting errors and confusion. Author: Marcelo Vanzin Closes #450 from vanzin/event-file-2 and squashes the following commits: 592cdb3 [Marcelo Vanzin] Honor default fs name when initializing event logger. --- .../scala/org/apache/spark/SparkContext.scala | 48 +++++++++---------- .../scheduler/EventLoggingListener.scala | 9 +++- .../org/apache/spark/util/FileLogger.scala | 17 ++++--- 3 files changed, 41 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 25ca650a3a37e..c14dce8273bc1 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -216,10 +216,33 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] val ui = new SparkUI(this) ui.bind() + /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ + val hadoopConfiguration: Configuration = { + val env = SparkEnv.get + val hadoopConf = SparkHadoopUtil.get.newConfiguration() + // Explicitly check for S3 environment variables + if (System.getenv("AWS_ACCESS_KEY_ID") != null && + System.getenv("AWS_SECRET_ACCESS_KEY") != null) { + hadoopConf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) + hadoopConf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) + hadoopConf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) + hadoopConf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) + } + // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar" + conf.getAll.foreach { case (key, value) => + if (key.startsWith("spark.hadoop.")) { + hadoopConf.set(key.substring("spark.hadoop.".length), value) + } + } + val bufferSize = conf.get("spark.buffer.size", "65536") + hadoopConf.set("io.file.buffer.size", bufferSize) + hadoopConf + } + // Optionally log Spark events private[spark] val eventLogger: Option[EventLoggingListener] = { if (conf.getBoolean("spark.eventLog.enabled", false)) { - val logger = new EventLoggingListener(appName, conf) + val logger = new EventLoggingListener(appName, conf, hadoopConfiguration) logger.start() listenerBus.addListener(logger) Some(logger) @@ -294,29 +317,6 @@ class SparkContext(config: SparkConf) extends Logging { postEnvironmentUpdate() postApplicationStart() - /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ - val hadoopConfiguration: Configuration = { - val env = SparkEnv.get - val hadoopConf = SparkHadoopUtil.get.newConfiguration() - // Explicitly check for S3 environment variables - if (System.getenv("AWS_ACCESS_KEY_ID") != null && - System.getenv("AWS_SECRET_ACCESS_KEY") != null) { - hadoopConf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) - hadoopConf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) - hadoopConf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) - hadoopConf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) - } - // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar" - conf.getAll.foreach { case (key, value) => - if (key.startsWith("spark.hadoop.")) { - hadoopConf.set(key.substring("spark.hadoop.".length), value) - } - } - val bufferSize = conf.get("spark.buffer.size", "65536") - hadoopConf.set("io.file.buffer.size", bufferSize) - hadoopConf - } - private[spark] var checkpointDir: Option[String] = None // Thread Local variable that can be used by users to pass information down the stack diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index b983c16af14f4..2fe65cd944b67 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import scala.collection.mutable +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.json4s.jackson.JsonMethods._ @@ -36,7 +37,10 @@ import org.apache.spark.util.{FileLogger, JsonProtocol} * spark.eventLog.dir - Path to the directory in which events are logged. * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams */ -private[spark] class EventLoggingListener(appName: String, conf: SparkConf) +private[spark] class EventLoggingListener( + appName: String, + conf: SparkConf, + hadoopConfiguration: Configuration) extends SparkListener with Logging { import EventLoggingListener._ @@ -49,7 +53,8 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) val logDir = logBaseDir + "/" + name private val logger = - new FileLogger(logDir, conf, outputBufferSize, shouldCompress, shouldOverwrite) + new FileLogger(logDir, conf, hadoopConfiguration, outputBufferSize, shouldCompress, + shouldOverwrite) /** * Begin logging events. diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 7d58d1c765180..7d47b2a72aff7 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -22,7 +22,8 @@ import java.net.URI import java.text.SimpleDateFormat import java.util.Date -import org.apache.hadoop.fs.{FSDataOutputStream, Path} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} import org.apache.spark.{Logging, SparkConf} import org.apache.spark.io.CompressionCodec @@ -37,7 +38,8 @@ import org.apache.spark.io.CompressionCodec */ private[spark] class FileLogger( logDir: String, - conf: SparkConf = new SparkConf, + conf: SparkConf, + hadoopConfiguration: Configuration, outputBufferSize: Int = 8 * 1024, // 8 KB compress: Boolean = false, overwrite: Boolean = true) @@ -85,19 +87,20 @@ private[spark] class FileLogger( private def createWriter(fileName: String): PrintWriter = { val logPath = logDir + "/" + fileName val uri = new URI(logPath) + val defaultFs = FileSystem.getDefaultUri(hadoopConfiguration).getScheme + val isDefaultLocal = (defaultFs == null || defaultFs == "file") /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). * Therefore, for local files, use FileOutputStream instead. */ - val dstream = uri.getScheme match { - case "file" | null => + val dstream = + if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { // Second parameter is whether to append new FileOutputStream(uri.getPath, !overwrite) - - case _ => + } else { val path = new Path(logPath) hadoopDataStream = Some(fileSystem.create(path, overwrite)) hadoopDataStream.get - } + } val bstream = new BufferedOutputStream(dstream, outputBufferSize) val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream From 432201c7ee9e1ea1d70a6418cbad1c5ad2653ed3 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Wed, 23 Apr 2014 16:52:49 -0700 Subject: [PATCH 145/641] SPARK-1582 Invoke Thread.interrupt() when cancelling jobs Sometimes executor threads are blocked waiting for IO or monitors, and the current implementation of job cancellation may never recover these threads. By simply invoking Thread.interrupt() during cancellation, we can often safely unblock the threads and use them for subsequent work. Note that this feature must remain optional for now because of a bug in HDFS where Thread.interrupt() may cause nodes to be marked as permanently dead (as the InterruptedException is reinterpreted as an IOException during communication with some node). Author: Aaron Davidson Closes #498 from aarondav/cancel and squashes the following commits: e52b829 [Aaron Davidson] Don't use job.properties when null 82f78bb [Aaron Davidson] Update DAGSchedulerSuite b67f472 [Aaron Davidson] Add comment on why interruptOnCancel is in setJobGroup 4cb9fd6 [Aaron Davidson] SPARK-1582 Invoke Thread.interrupt() when cancelling jobs --- .../scala/org/apache/spark/SparkContext.scala | 15 ++++++++++++++- .../org/apache/spark/api/python/PythonRDD.scala | 1 - .../executor/CoarseGrainedExecutorBackend.scala | 4 ++-- .../org/apache/spark/executor/Executor.scala | 10 +++++----- .../spark/executor/MesosExecutorBackend.scala | 3 ++- .../org/apache/spark/scheduler/DAGScheduler.scala | 6 +++++- .../apache/spark/scheduler/SchedulerBackend.scala | 3 ++- .../scala/org/apache/spark/scheduler/Task.scala | 12 ++++++++++-- .../apache/spark/scheduler/TaskScheduler.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 4 ++-- .../org/apache/spark/scheduler/TaskSet.scala | 4 ++-- .../cluster/CoarseGrainedClusterMessage.scala | 3 ++- .../cluster/CoarseGrainedSchedulerBackend.scala | 8 ++++---- .../spark/scheduler/local/LocalBackend.scala | 10 +++++----- .../spark/scheduler/DAGSchedulerSuite.scala | 2 +- 15 files changed, 57 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c14dce8273bc1..dcb6b6824b0a6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -381,16 +381,27 @@ class SparkContext(config: SparkConf) extends Logging { * // In a separate thread: * sc.cancelJobGroup("some_job_to_cancel") * }}} + * + * If interruptOnCancel is set to true for the job group, then job cancellation will result + * in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure + * that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, + * where HDFS may respond to Thread.interrupt() by marking nodes as dead. */ - def setJobGroup(groupId: String, description: String) { + def setJobGroup(groupId: String, description: String, interruptOnCancel: Boolean = false) { setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, description) setLocalProperty(SparkContext.SPARK_JOB_GROUP_ID, groupId) + // Note: Specifying interruptOnCancel in setJobGroup (rather than cancelJobGroup) avoids + // changing several public APIs and allows Spark cancellations outside of the cancelJobGroup + // APIs to also take advantage of this property (e.g., internal job failures or canceling from + // JobProgressTab UI) on a per-job basis. + setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, interruptOnCancel.toString) } /** Clear the current thread's job group ID and its description. */ def clearJobGroup() { setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, null) setLocalProperty(SparkContext.SPARK_JOB_GROUP_ID, null) + setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, null) } // Post init @@ -1244,6 +1255,8 @@ object SparkContext extends Logging { private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id" + private[spark] val SPARK_JOB_INTERRUPT_ON_CANCEL = "spark.job.interruptOnCancel" + private[spark] val SPARK_UNKNOWN_USER = "" implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { 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 8a843fbb0e66f..0d71fdbb03ec6 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 @@ -169,7 +169,6 @@ private[spark] class PythonRDD[T: ClassTag]( val update = new Array[Byte](updateLen) stream.readFully(update) accumulator += Collections.singletonList(update) - } Array.empty[Byte] } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 6327ac01663f6..9ac7365f47f9f 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -69,12 +69,12 @@ private[spark] class CoarseGrainedExecutorBackend( executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask) } - case KillTask(taskId, _) => + case KillTask(taskId, _, interruptThread) => if (executor == null) { logError("Received KillTask command but executor was null") System.exit(1) } else { - executor.killTask(taskId) + executor.killTask(taskId, interruptThread) } case x: DisassociatedEvent => 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 2bfb9c387e1c9..914bc205cebe2 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -136,10 +136,10 @@ private[spark] class Executor( threadPool.execute(tr) } - def killTask(taskId: Long) { + def killTask(taskId: Long, interruptThread: Boolean) { val tr = runningTasks.get(taskId) if (tr != null) { - tr.kill() + tr.kill(interruptThread) } } @@ -166,11 +166,11 @@ private[spark] class Executor( @volatile private var killed = false @volatile private var task: Task[Any] = _ - def kill() { + def kill(interruptThread: Boolean) { logInfo("Executor is trying to kill task " + taskId) killed = true if (task != null) { - task.kill() + task.kill(interruptThread) } } @@ -257,7 +257,7 @@ private[spark] class Executor( execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) } - case TaskKilledException => { + case TaskKilledException | _: InterruptedException if task.killed => { logInfo("Executor killed task " + taskId) execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) } diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 6fc702fdb1512..64e24506e8038 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -76,7 +76,8 @@ private[spark] class MesosExecutorBackend if (executor == null) { logError("Received KillTask but executor was null") } else { - executor.killTask(t.getValue.toLong) + // TODO: Determine the 'interruptOnCancel' property set for the given job. + executor.killTask(t.getValue.toLong, interruptThread = false) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c6cbf14e20069..dbde9b591dccc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1055,6 +1055,10 @@ class DAGScheduler( val error = new SparkException(failureReason) job.listener.jobFailed(error) + val shouldInterruptThread = + if (job.properties == null) false + else job.properties.getProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "false").toBoolean + // Cancel all independent, running stages. val stages = jobIdToStageIds(job.jobId) if (stages.isEmpty) { @@ -1073,7 +1077,7 @@ class DAGScheduler( // This is the only job that uses this stage, so fail the stage if it is running. val stage = stageIdToStage(stageId) if (runningStages.contains(stage)) { - taskScheduler.cancelTasks(stageId) + taskScheduler.cancelTasks(stageId, shouldInterruptThread) val stageInfo = stageToInfos(stage) stageInfo.stageFailed(failureReason) listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index f1924a4573b21..6a6d8e609bc39 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -28,5 +28,6 @@ private[spark] trait SchedulerBackend { def reviveOffers(): Unit def defaultParallelism(): Int - def killTask(taskId: Long, executorId: String): Unit = throw new UnsupportedOperationException + def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = + throw new UnsupportedOperationException } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index a8bcb7dfe2f3c..2ca3479c80efc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -44,8 +44,9 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex final def run(attemptId: Long): T = { context = new TaskContext(stageId, partitionId, attemptId, runningLocally = false) + taskThread = Thread.currentThread() if (_killed) { - kill() + kill(interruptThread = false) } runTask(context) } @@ -62,6 +63,9 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex // Task context, to be initialized in run(). @transient protected var context: TaskContext = _ + // The actual Thread on which the task is running, if any. Initialized in run(). + @volatile @transient private var taskThread: Thread = _ + // A flag to indicate whether the task is killed. This is used in case context is not yet // initialized when kill() is invoked. @volatile @transient private var _killed = false @@ -75,12 +79,16 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex * Kills a task by setting the interrupted flag to true. This relies on the upper level Spark * code and user code to properly handle the flag. This function should be idempotent so it can * be called multiple times. + * If interruptThread is true, we will also call Thread.interrupt() on the Task's executor thread. */ - def kill() { + def kill(interruptThread: Boolean) { _killed = true if (context != null) { context.interrupted = true } + if (interruptThread && taskThread != null) { + taskThread.interrupt() + } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 92616c997e20c..819c35257b5a7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -47,7 +47,7 @@ private[spark] trait TaskScheduler { def submitTasks(taskSet: TaskSet): Unit // Cancel a stage. - def cancelTasks(stageId: Int) + def cancelTasks(stageId: Int, interruptThread: Boolean) // Set the DAG scheduler for upcalls. This is guaranteed to be set before submitTasks is called. def setDAGScheduler(dagScheduler: DAGScheduler): Unit diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index fe72ab3e43146..be19d9b8854c8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -170,7 +170,7 @@ private[spark] class TaskSchedulerImpl( backend.reviveOffers() } - override def cancelTasks(stageId: Int): Unit = synchronized { + override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized { logInfo("Cancelling stage " + stageId) activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) => // There are two possible cases here: @@ -181,7 +181,7 @@ private[spark] class TaskSchedulerImpl( // simply abort the stage. tsm.runningTasksSet.foreach { tid => val execId = taskIdToExecutorId(tid) - backend.killTask(tid, execId) + backend.killTask(tid, execId, interruptThread) } tsm.abort("Stage %s cancelled".format(stageId)) logInfo("Stage %d was cancelled".format(stageId)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala index 03bf76083761f..613fa7850bb25 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala @@ -31,8 +31,8 @@ private[spark] class TaskSet( val properties: Properties) { val id: String = stageId + "." + attempt - def kill() { - tasks.foreach(_.kill()) + def kill(interruptThread: Boolean) { + tasks.foreach(_.kill(interruptThread)) } override def toString: String = "TaskSet " + id diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 4a9a1659d8254..ddbc74e82ac49 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -30,7 +30,8 @@ private[spark] object CoarseGrainedClusterMessages { // Driver to executors case class LaunchTask(task: TaskDescription) extends CoarseGrainedClusterMessage - case class KillTask(taskId: Long, executor: String) extends CoarseGrainedClusterMessage + case class KillTask(taskId: Long, executor: String, interruptThread: Boolean) + extends CoarseGrainedClusterMessage case class RegisteredExecutor(sparkProperties: Seq[(String, String)]) extends CoarseGrainedClusterMessage diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 7bfc30b4208a3..a6d6b3d26a3c6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -101,8 +101,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A case ReviveOffers => makeOffers() - case KillTask(taskId, executorId) => - executorActor(executorId) ! KillTask(taskId, executorId) + case KillTask(taskId, executorId, interruptThread) => + executorActor(executorId) ! KillTask(taskId, executorId, interruptThread) case StopDriver => sender ! true @@ -207,8 +207,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A driverActor ! ReviveOffers } - override def killTask(taskId: Long, executorId: String) { - driverActor ! KillTask(taskId, executorId) + override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { + driverActor ! KillTask(taskId, executorId, interruptThread) } override def defaultParallelism(): Int = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 16e2f5cf3076d..43f0e18a0cbe0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -30,7 +30,7 @@ private case class ReviveOffers() private case class StatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) -private case class KillTask(taskId: Long) +private case class KillTask(taskId: Long, interruptThread: Boolean) /** * Calls to LocalBackend are all serialized through LocalActor. Using an actor makes the calls on @@ -61,8 +61,8 @@ private[spark] class LocalActor( reviveOffers() } - case KillTask(taskId) => - executor.killTask(taskId) + case KillTask(taskId, interruptThread) => + executor.killTask(taskId, interruptThread) } def reviveOffers() { @@ -99,8 +99,8 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: override def defaultParallelism() = totalCores - override def killTask(taskId: Long, executorId: String) { - localActor ! KillTask(taskId) + override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { + localActor ! KillTask(taskId, interruptThread) } override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 35a7ac9d049c2..ff69eb7e53f8e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -58,7 +58,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) taskSets += taskSet } - override def cancelTasks(stageId: Int) { + override def cancelTasks(stageId: Int, interruptThread: Boolean) { cancelledStages += stageId } override def setDAGScheduler(dagScheduler: DAGScheduler) = {} From 640f9a0efefd42cff86aecd4878a3a57f5ae85fa Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 23 Apr 2014 17:03:54 -0700 Subject: [PATCH 146/641] [SPARK-1540] Add an optional Ordering parameter to PairRDDFunctions. In https://issues.apache.org/jira/browse/SPARK-1540 we'd like to look at Spark's API to see if we can take advantage of Comparable keys in more places, which will make external spilling more efficient. This PR is a first step towards that that shows how to pass an Ordering when available and still continue functioning otherwise. It does this using a new implicit parameter with a default value of null. The API is currently only in Scala -- in Java we'd have to add new versions of mapToPair and such that take a Comparator, or a new method to add a "type hint" to an RDD. We can address those later though. Unfortunately requiring all keys to be Comparable would not work without requiring RDDs in general to contain only Comparable types. The reason is that methods such as distinct() and intersection() do a shuffle, but should be usable on RDDs of any type. So ordering will have to remain an optimization for the types that can be ordered. I think this isn't a horrible outcome though because one of the nice things about Spark's API is that it works on objects of *any* type, without requiring you to specify a schema or implement Writable or stuff like that. Author: Matei Zaharia This patch had conflicts when merged, resolved by Committer: Reynold Xin Closes #487 from mateiz/ordered-keys and squashes the following commits: bd565f6 [Matei Zaharia] Pass an Ordering to only one version of groupBy because the Scala language spec doesn't allow having an optional parameter on all of them (this was only compiling in Scala 2.10 due to a bug). 4629965 [Matei Zaharia] Add tests for other versions of groupBy 3beae85 [Matei Zaharia] Added a test for implicit orderings 80b7a3b [Matei Zaharia] Add an optional Ordering parameter to PairRDDFunctions. --- .../scala/org/apache/spark/SparkContext.scala | 4 +- .../apache/spark/rdd/PairRDDFunctions.scala | 29 +++++----- .../main/scala/org/apache/spark/rdd/RDD.scala | 46 ++++++++------- .../spark/rdd/SequenceFileRDDFunctions.scala | 4 +- .../apache/spark/ImplicitOrderingSuite.scala | 57 +++++++++++++++++++ .../spark/streaming/StreamingContext.scala | 5 +- .../spark/streaming/dstream/DStream.scala | 10 ++-- .../dstream/PairDStreamFunctions.scala | 18 +++--- 8 files changed, 124 insertions(+), 49 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index dcb6b6824b0a6..e9d2f5757963d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1281,8 +1281,10 @@ object SparkContext extends Logging { // TODO: Add AccumulatorParams for other types, e.g. lists and strings - implicit def rddToPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = + implicit def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { new PairRDDFunctions(rdd) + } implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = new AsyncRDDActions(rdd) 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 d250bef6aad0f..d2b9ee427656b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -52,11 +52,12 @@ import org.apache.spark.util.SerializableHyperLogLog * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ -class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) +class PairRDDFunctions[K, V](self: RDD[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) extends Logging with SparkHadoopMapReduceUtil - with Serializable { - + with Serializable +{ /** * Generic function to combine the elements for each key using a custom set of aggregation * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C @@ -77,7 +78,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) mapSideCombine: Boolean = true, serializer: Serializer = null): RDD[(K, C)] = { require(mergeCombiners != null, "mergeCombiners must be defined") // required as of Spark 0.9.0 - if (getKeyClass().isArray) { + if (keyClass.isArray) { if (mapSideCombine) { throw new SparkException("Cannot use map-side combining with array keys.") } @@ -170,7 +171,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def reduceByKeyLocally(func: (V, V) => V): Map[K, V] = { - if (getKeyClass().isArray) { + if (keyClass.isArray) { throw new SparkException("reduceByKeyLocally() does not support array keys") } @@ -288,7 +289,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Return a copy of the RDD partitioned using the specified partitioner. */ def partitionBy(partitioner: Partitioner): RDD[(K, V)] = { - if (getKeyClass().isArray && partitioner.isInstanceOf[HashPartitioner]) { + if (keyClass.isArray && partitioner.isInstanceOf[HashPartitioner]) { throw new SparkException("Default partitioner cannot partition array keys.") } if (self.partitioner == partitioner) self else new ShuffledRDD[K, V, (K, V)](self, partitioner) @@ -458,7 +459,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner) : RDD[(K, (Iterable[V], Iterable[W]))] = { - if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { + if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other), partitioner) @@ -473,7 +474,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { - if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { + if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) @@ -573,7 +574,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * supporting the key and value types K and V in this RDD. */ def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) { - saveAsHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]]) + saveAsHadoopFile(path, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } /** @@ -584,7 +585,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def saveAsHadoopFile[F <: OutputFormat[K, V]]( path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassTag[F]) { val runtimeClass = fm.runtimeClass - saveAsHadoopFile(path, getKeyClass, getValueClass, runtimeClass.asInstanceOf[Class[F]], codec) + saveAsHadoopFile(path, keyClass, valueClass, runtimeClass.asInstanceOf[Class[F]], codec) } /** @@ -592,7 +593,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD. */ def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) { - saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]]) + saveAsNewAPIHadoopFile(path, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } /** @@ -782,7 +783,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def values: RDD[V] = self.map(_._2) - private[spark] def getKeyClass() = implicitly[ClassTag[K]].runtimeClass + private[spark] def keyClass: Class[_] = kt.runtimeClass + + private[spark] def valueClass: Class[_] = vt.runtimeClass - private[spark] def getValueClass() = implicitly[ClassTag[V]].runtimeClass + private[spark] def keyOrdering: Option[Ordering[K]] = Option(ord) } 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 596dcb84db7bf..6c897cc03b641 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -284,7 +284,7 @@ abstract class RDD[T: ClassTag]( /** * Return a new RDD containing the distinct elements in this RDD. */ - def distinct(numPartitions: Int): RDD[T] = + def distinct(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = map(x => (x, null)).reduceByKey((x, y) => x, numPartitions).map(_._1) /** @@ -301,7 +301,7 @@ abstract class RDD[T: ClassTag]( * If you are decreasing the number of partitions in this RDD, consider using `coalesce`, * which can avoid performing a shuffle. */ - def repartition(numPartitions: Int): RDD[T] = { + def repartition(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = { coalesce(numPartitions, shuffle = true) } @@ -325,7 +325,8 @@ abstract class RDD[T: ClassTag]( * coalesce(1000, shuffle = true) will result in 1000 partitions with the * data distributed using a hash partitioner. */ - def coalesce(numPartitions: Int, shuffle: Boolean = false): RDD[T] = { + def coalesce(numPartitions: Int, shuffle: Boolean = false)(implicit ord: Ordering[T] = null) + : RDD[T] = { if (shuffle) { // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( @@ -424,10 +425,11 @@ abstract class RDD[T: ClassTag]( * * Note that this method performs a shuffle internally. */ - def intersection(other: RDD[T]): RDD[T] = + def intersection(other: RDD[T]): RDD[T] = { this.map(v => (v, null)).cogroup(other.map(v => (v, null))) .filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty } .keys + } /** * Return the intersection of this RDD and another one. The output will not contain any duplicate @@ -437,10 +439,12 @@ abstract class RDD[T: ClassTag]( * * @param partitioner Partitioner to use for the resulting RDD */ - def intersection(other: RDD[T], partitioner: Partitioner): RDD[T] = + def intersection(other: RDD[T], partitioner: Partitioner)(implicit ord: Ordering[T] = null) + : RDD[T] = { this.map(v => (v, null)).cogroup(other.map(v => (v, null)), partitioner) .filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty } .keys + } /** * Return the intersection of this RDD and another one. The output will not contain any duplicate @@ -450,10 +454,11 @@ abstract class RDD[T: ClassTag]( * * @param numPartitions How many partitions to use in the resulting RDD */ - def intersection(other: RDD[T], numPartitions: Int): RDD[T] = + def intersection(other: RDD[T], numPartitions: Int): RDD[T] = { this.map(v => (v, null)).cogroup(other.map(v => (v, null)), new HashPartitioner(numPartitions)) .filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty } .keys + } /** * Return an RDD created by coalescing all elements within each partition into an array. @@ -467,22 +472,25 @@ abstract class RDD[T: ClassTag]( def cartesian[U: ClassTag](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other) /** - * Return an RDD of grouped items. + * Return an RDD of grouped items. Each group consists of a key and a sequence of elements + * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K): RDD[(K, Iterable[T])] = + def groupBy[K](f: T => K)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = groupBy[K](f, defaultPartitioner(this)) /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Iterable[T])] = + def groupBy[K](f: T => K, numPartitions: Int)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = groupBy(f, new HashPartitioner(numPartitions)) /** - * Return an RDD of grouped items. + * Return an RDD of grouped items. Each group consists of a key and a sequence of elements + * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Iterable[T])] = { + def groupBy[K](f: T => K, p: Partitioner)(implicit kt: ClassTag[K], ord: Ordering[K] = null) + : RDD[(K, Iterable[T])] = { val cleanF = sc.clean(f) this.map(t => (cleanF(t), t)).groupByKey(p) } @@ -739,7 +747,7 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD with the elements from `this` that are not in `other`. */ - def subtract(other: RDD[T], p: Partitioner): RDD[T] = { + def subtract(other: RDD[T], p: Partitioner)(implicit ord: Ordering[T] = null): RDD[T] = { if (partitioner == Some(p)) { // Our partitioner knows how to handle T (which, since we have a partitioner, is // really (K, V)) so make a new Partitioner that will de-tuple our fake tuples @@ -847,7 +855,7 @@ abstract class RDD[T: ClassTag]( * Return the count of each unique value in this RDD as a map of (value, count) pairs. The final * combine step happens locally on the master, equivalent to running a single reduce task. */ - def countByValue(): Map[T, Long] = { + def countByValue()(implicit ord: Ordering[T] = null): Map[T, Long] = { if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValue() does not support arrays") } @@ -877,10 +885,10 @@ abstract class RDD[T: ClassTag]( * Approximate version of countByValue(). */ @Experimental - def countByValueApprox( - timeout: Long, - confidence: Double = 0.95 - ): PartialResult[Map[T, BoundedDouble]] = { + def countByValueApprox(timeout: Long, confidence: Double = 0.95) + (implicit ord: Ordering[T] = null) + : PartialResult[Map[T, BoundedDouble]] = + { if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValueApprox() does not support arrays") } @@ -1030,13 +1038,13 @@ abstract class RDD[T: ClassTag]( * 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) + 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) + 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/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala index 7df9a2960d8a5..9a1efc83cbe6a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -68,8 +68,8 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag val keyClass = getWritableClass[K] val valueClass = getWritableClass[V] - val convertKey = !classOf[Writable].isAssignableFrom(self.getKeyClass) - val convertValue = !classOf[Writable].isAssignableFrom(self.getValueClass) + val convertKey = !classOf[Writable].isAssignableFrom(self.keyClass) + val convertValue = !classOf[Writable].isAssignableFrom(self.valueClass) logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + valueClass.getSimpleName + ")" ) diff --git a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala new file mode 100644 index 0000000000000..4bd889135631b --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala @@ -0,0 +1,57 @@ +/* + * 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 + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext._ + +class ImplicitOrderingSuite extends FunSuite with LocalSparkContext { + class NonOrderedClass {} + + class ComparableClass extends Comparable[ComparableClass] { + override def compareTo(o: ComparableClass): Int = ??? + } + + class OrderedClass extends Ordered[OrderedClass] { + override def compare(o: OrderedClass): Int = ??? + } + + // Tests that PairRDDFunctions grabs an implicit Ordering in various cases where it should. + test("basic inference of Orderings"){ + sc = new SparkContext("local", "test") + val rdd = sc.parallelize(1 to 10) + + // Infer orderings after basic maps to particular types + assert(rdd.map(x => (x, x)).keyOrdering.isDefined) + assert(rdd.map(x => (1, x)).keyOrdering.isDefined) + assert(rdd.map(x => (x.toString, x)).keyOrdering.isDefined) + assert(rdd.map(x => (null, x)).keyOrdering.isDefined) + assert(rdd.map(x => (new NonOrderedClass, x)).keyOrdering.isEmpty) + assert(rdd.map(x => (new ComparableClass, x)).keyOrdering.isDefined) + assert(rdd.map(x => (new OrderedClass, x)).keyOrdering.isDefined) + + // Infer orderings for other RDD methods + assert(rdd.groupBy(x => x).keyOrdering.isDefined) + assert(rdd.groupBy(x => new NonOrderedClass).keyOrdering.isEmpty) + assert(rdd.groupBy(x => new ComparableClass).keyOrdering.isDefined) + assert(rdd.groupBy(x => new OrderedClass).keyOrdering.isDefined) + assert(rdd.groupBy((x: Int) => x, 5).keyOrdering.isDefined) + assert(rdd.groupBy((x: Int) => x, new HashPartitioner(5)).keyOrdering.isDefined) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 9ba6e02229aaa..1c89543058211 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -495,7 +495,10 @@ class StreamingContext private[streaming] ( object StreamingContext extends Logging { - implicit def toPairDStreamFunctions[K: ClassTag, V: ClassTag](stream: DStream[(K,V)]) = { + private[streaming] val DEFAULT_CLEANER_TTL = 3600 + + implicit def toPairDStreamFunctions[K, V](stream: DStream[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { new PairDStreamFunctions[K, V](stream) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index a7e5215437e54..d393cc03cb33e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -488,7 +488,8 @@ abstract class DStream[T: ClassTag] ( * the RDDs with `numPartitions` partitions (Spark's default number of partitions if * `numPartitions` not specified). */ - def countByValue(numPartitions: Int = ssc.sc.defaultParallelism): DStream[(T, Long)] = + def countByValue(numPartitions: Int = ssc.sc.defaultParallelism)(implicit ord: Ordering[T] = null) + : DStream[(T, Long)] = this.map(x => (x, 1L)).reduceByKey((x: Long, y: Long) => x + y, numPartitions) /** @@ -686,9 +687,10 @@ abstract class DStream[T: ClassTag] ( def countByValueAndWindow( windowDuration: Duration, slideDuration: Duration, - numPartitions: Int = ssc.sc.defaultParallelism - ): DStream[(T, Long)] = { - + numPartitions: Int = ssc.sc.defaultParallelism) + (implicit ord: Ordering[T] = null) + : DStream[(T, Long)] = + { this.map(x => (x, 1L)).reduceByKeyAndWindow( (x: Long, y: Long) => x + y, (x: Long, y: Long) => x - y, diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index 354bc132dcdc0..826bf39e860e1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -37,13 +37,13 @@ import org.apache.spark.streaming.{Time, Duration} * Import `org.apache.spark.streaming.StreamingContext._` at the top of your program to use * these functions. */ -class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) - extends Serializable { - +class PairDStreamFunctions[K, V](self: DStream[(K,V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K]) + extends Serializable +{ private[streaming] def ssc = self.ssc - private[streaming] def defaultPartitioner(numPartitions: Int = self.ssc.sc.defaultParallelism) - = { + private[streaming] def defaultPartitioner(numPartitions: Int = self.ssc.sc.defaultParallelism) = { new HashPartitioner(numPartitions) } @@ -576,7 +576,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) prefix: String, suffix: String )(implicit fm: ClassTag[F]) { - saveAsHadoopFiles(prefix, suffix, getKeyClass, getValueClass, + saveAsHadoopFiles(prefix, suffix, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } @@ -607,7 +607,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) prefix: String, suffix: String )(implicit fm: ClassTag[F]) { - saveAsNewAPIHadoopFiles(prefix, suffix, getKeyClass, getValueClass, + saveAsNewAPIHadoopFiles(prefix, suffix, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } @@ -630,7 +630,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) self.foreachRDD(saveFunc) } - private def getKeyClass() = implicitly[ClassTag[K]].runtimeClass + private def keyClass: Class[_] = kt.runtimeClass - private def getValueClass() = implicitly[ClassTag[V]].runtimeClass + private def valueClass: Class[_] = vt.runtimeClass } From 4b2bab1d08a6b790be94717bbdd643d896d85c16 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 23 Apr 2014 21:59:33 -0700 Subject: [PATCH 147/641] [Hot Fix #469] Fix flaky test in SparkListenerSuite The two modified tests may fail if the race condition does not bid in our favor... Author: Andrew Or Closes #516 from andrewor14/stage-info-test-fix and squashes the following commits: b4b6100 [Andrew Or] Add/replace missing waitUntilEmpty() calls to listener bus --- .../org/apache/spark/scheduler/SparkListenerSuite.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index ab139175e00eb..ba048ced32a93 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -50,9 +50,9 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } assert(counter.count === 0) - // Starting listener bus should flush all buffered events (asynchronously, hence the sleep) + // Starting listener bus should flush all buffered events bus.start() - Thread.sleep(1000) + assert(bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(counter.count === 5) // After listener bus has stopped, posting events should not increment counter @@ -177,6 +177,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc listener.stageInfos.clear() rdd3.count() + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {2} // Shuffle map stage + result stage val stageInfo3 = listener.stageInfos.keys.find(_.stageId == 2).get stageInfo3.rddInfos.size should be {2} // ShuffledRDD, MapPartitionsRDD From d485eecb7233dd339ae85a6f58f1c0686dd2037d Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Wed, 23 Apr 2014 22:00:22 -0700 Subject: [PATCH 148/641] Update Java api for setJobGroup with interruptOnCancel Also adds a unit test. Author: Aaron Davidson Closes #522 from aarondav/cancel2 and squashes the following commits: 565c253 [Aaron Davidson] Update Java api for setJobGroup with interruptOnCancel 65b33d8 [Aaron Davidson] Add unit test for Thread interruption on cancellation --- .../spark/api/java/JavaSparkContext.scala | 15 ++++++++ .../apache/spark/JobCancellationSuite.scala | 36 +++++++++++++++++-- 2 files changed, 49 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index bda9272b43393..8b95cda511643 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -570,6 +570,21 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * // In a separate thread: * sc.cancelJobGroup("some_job_to_cancel"); * }}} + * + * If interruptOnCancel is set to true for the job group, then job cancellation will result + * in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure + * that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, + * where HDFS may respond to Thread.interrupt() by marking nodes as dead. + */ + def setJobGroup(groupId: String, description: String, interruptOnCancel: Boolean): Unit = + sc.setJobGroup(groupId, description, interruptOnCancel) + + /** + * Assigns a group ID to all the jobs started by this thread until the group ID is set to a + * different value or cleared. + * + * @see `setJobGroup(groupId: String, description: String, interruptThread: Boolean)`. + * This method sets interruptOnCancel to false. */ def setJobGroup(groupId: String, description: String): Unit = sc.setJobGroup(groupId, description) diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 7a39d1af9e2d5..16cfdf11c4a38 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -21,7 +21,7 @@ import java.util.concurrent.Semaphore import scala.concurrent.Await import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent.duration.Duration +import scala.concurrent.duration._ import scala.concurrent.future import org.scalatest.{BeforeAndAfter, FunSuite} @@ -101,18 +101,50 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.count() } + // Block until both tasks of job A have started and cancel job A. + sem.acquire(2) + sc.clearJobGroup() val jobB = sc.parallelize(1 to 100, 2).countAsync() + sc.cancelJobGroup("jobA") + val e = intercept[SparkException] { Await.result(jobA, Duration.Inf) } + assert(e.getMessage contains "cancel") + + // Once A is cancelled, job B should finish fairly quickly. + assert(jobB.get() === 100) + } + + + test("job group with interruption") { + sc = new SparkContext("local[2]", "test") + + // Add a listener to release the semaphore once any tasks are launched. + val sem = new Semaphore(0) + sc.addSparkListener(new SparkListener { + override def onTaskStart(taskStart: SparkListenerTaskStart) { + sem.release() + } + }) + + // jobA is the one to be cancelled. + val jobA = future { + sc.setJobGroup("jobA", "this is a job to be cancelled", interruptOnCancel = true) + sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(100000); i }.count() + } // Block until both tasks of job A have started and cancel job A. sem.acquire(2) + + sc.clearJobGroup() + val jobB = sc.parallelize(1 to 100, 2).countAsync() sc.cancelJobGroup("jobA") - val e = intercept[SparkException] { Await.result(jobA, Duration.Inf) } + val e = intercept[SparkException] { Await.result(jobA, 5.seconds) } assert(e.getMessage contains "cancel") // Once A is cancelled, job B should finish fairly quickly. assert(jobB.get() === 100) } + /* test("two jobs sharing the same stage") { // sem1: make sure cancel is issued after some tasks are launched From 1d6abe3a4b58f28fc4e0e690e02c19b2568ce1ee Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 23 Apr 2014 22:01:13 -0700 Subject: [PATCH 149/641] Mark all fields of EdgePartition, Graph, and GraphOps transient These classes are only serializable to work around closure capture, so their fields should all be marked `@transient` to avoid wasteful serialization. This PR supersedes apache/spark#519 and fixes the same bug. Author: Ankur Dave Closes #520 from ankurdave/graphx-transient and squashes the following commits: 6431760 [Ankur Dave] Mark all fields of EdgePartition, Graph, and GraphOps `@transient` --- .../src/main/scala/org/apache/spark/graphx/Graph.scala | 6 +++--- .../main/scala/org/apache/spark/graphx/GraphOps.scala | 10 +++++----- .../org/apache/spark/graphx/impl/EdgePartition.scala | 8 ++++---- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 45349692cbf6c..50395868902dc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -46,7 +46,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * @note vertex ids are unique. * @return an RDD containing the vertices in this graph */ - val vertices: VertexRDD[VD] + @transient val vertices: VertexRDD[VD] /** * An RDD containing the edges and their associated attributes. The entries in the RDD contain @@ -59,7 +59,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * along with their vertex data. * */ - val edges: EdgeRDD[ED] + @transient val edges: EdgeRDD[ED] /** * An RDD containing the edge triplets, which are edges along with the vertex data associated with @@ -77,7 +77,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * val numInvalid = graph.triplets.map(e => if (e.src.data == e.dst.data) 1 else 0).sum * }}} */ - val triplets: RDD[EdgeTriplet[VD, ED]] + @transient val triplets: RDD[EdgeTriplet[VD, ED]] /** * Caches the vertices and edges associated with this graph at the specified storage level. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 5635287694ee2..4997fbc3cbcd8 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -34,28 +34,28 @@ import scala.util.Random class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Serializable { /** The number of edges in the graph. */ - lazy val numEdges: Long = graph.edges.count() + @transient lazy val numEdges: Long = graph.edges.count() /** The number of vertices in the graph. */ - lazy val numVertices: Long = graph.vertices.count() + @transient lazy val numVertices: Long = graph.vertices.count() /** * The in-degree of each vertex in the graph. * @note Vertices with no in-edges are not returned in the resulting RDD. */ - lazy val inDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.In) + @transient lazy val inDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.In) /** * The out-degree of each vertex in the graph. * @note Vertices with no out-edges are not returned in the resulting RDD. */ - lazy val outDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Out) + @transient lazy val outDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Out) /** * The degree of each vertex in the graph. * @note Vertices with no edges are not returned in the resulting RDD. */ - lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Either) + @transient lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Either) /** * Computes the neighboring vertex degrees. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index 2e05f5d4e4969..b7c472e905a9b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -34,10 +34,10 @@ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap */ private[graphx] class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag]( - val srcIds: Array[VertexId], - val dstIds: Array[VertexId], - val data: Array[ED], - val index: PrimitiveKeyOpenHashMap[VertexId, Int]) extends Serializable { + @transient val srcIds: Array[VertexId], + @transient val dstIds: Array[VertexId], + @transient val data: Array[ED], + @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int]) extends Serializable { /** * Reverse all the edges in this partition. From 6ab7578067e3bb78b64f99fd67c97e9607050ffe Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 23 Apr 2014 22:36:02 -0700 Subject: [PATCH 150/641] SPARK-1589: Fix the incorrect compare JIRA: https://issues.apache.org/jira/browse/SPARK-1589 Author: zsxwing Closes #508 from zsxwing/SPARK-1589 and squashes the following commits: 570c67a [zsxwing] SPARK-1589: Fix the incorrect compare --- .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 d2b9ee427656b..5efb4388f6c71 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -292,7 +292,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) if (keyClass.isArray && partitioner.isInstanceOf[HashPartitioner]) { throw new SparkException("Default partitioner cannot partition array keys.") } - if (self.partitioner == partitioner) self else new ShuffledRDD[K, V, (K, V)](self, partitioner) + if (self.partitioner == Some(partitioner)) { + self + } else { + new ShuffledRDD[K, V, (K, V)](self, partitioner) + } } /** From bb68f47745eec2954814d3da277a672d5cf89980 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Wed, 23 Apr 2014 22:47:59 -0700 Subject: [PATCH 151/641] [Fix #79] Replace Breakable For Loops By While Loops Author: Sandeep Closes #503 from techaddict/fix-79 and squashes the following commits: e3f6746 [Sandeep] Style changes 07a4f6b [Sandeep] for loop to While loop 0a6d8e9 [Sandeep] Breakable for loop to While loop --- .../spark/mllib/tree/DecisionTree.scala | 60 ++++++++++--------- 1 file changed, 31 insertions(+), 29 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 3019447ce4cd9..f68076f426259 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -17,8 +17,6 @@ package org.apache.spark.mllib.tree -import scala.util.control.Breaks._ - import org.apache.spark.annotation.Experimental import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ @@ -82,31 +80,34 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo * still survived the filters of the parent nodes. */ - // TODO: Convert for loop to while loop - breakable { - for (level <- 0 until maxDepth) { - - logDebug("#####################################") - logDebug("level = " + level) - logDebug("#####################################") - - // Find best split for all nodes at a level. - val splitsStatsForLevel = DecisionTree.findBestSplits(input, parentImpurities, strategy, - level, filters, splits, bins) - - for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { - // Extract info for nodes at the current level. - extractNodeInfo(nodeSplitStats, level, index, nodes) - // Extract info for nodes at the next lower level. - extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities, - filters) - logDebug("final best split = " + nodeSplitStats._1) - } - require(scala.math.pow(2, level) == splitsStatsForLevel.length) - // Check whether all the nodes at the current level at leaves. - val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0) - logDebug("all leaf = " + allLeaf) - if (allLeaf) break // no more tree construction + var level = 0 + var break = false + while (level < maxDepth && !break) { + + logDebug("#####################################") + logDebug("level = " + level) + logDebug("#####################################") + + // Find best split for all nodes at a level. + val splitsStatsForLevel = DecisionTree.findBestSplits(input, parentImpurities, strategy, + level, filters, splits, bins) + + for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { + // Extract info for nodes at the current level. + extractNodeInfo(nodeSplitStats, level, index, nodes) + // Extract info for nodes at the next lower level. + extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities, + filters) + logDebug("final best split = " + nodeSplitStats._1) + } + require(scala.math.pow(2, level) == splitsStatsForLevel.length) + // Check whether all the nodes at the current level at leaves. + val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0) + logDebug("all leaf = " + allLeaf) + if (allLeaf) { + break = true // no more tree construction + } else { + level += 1 } } @@ -146,8 +147,8 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo parentImpurities: Array[Double], filters: Array[List[Filter]]): Unit = { // 0 corresponds to the left child node and 1 corresponds to the right child node. - // TODO: Convert to while loop - for (i <- 0 to 1) { + var i = 0 + while (i <= 1) { // Calculate the index of the node from the node level and the index at the current level. val nodeIndex = scala.math.pow(2, level + 1).toInt - 1 + 2 * index + i if (level < maxDepth - 1) { @@ -166,6 +167,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo logDebug("Filter = " + filter) } } + i += 1 } } } From dd681f502eafe39cfb8a5a62ea2d28016ac6013d Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 23 Apr 2014 23:20:55 -0700 Subject: [PATCH 152/641] SPARK-1587 Fix thread leak mvn test fails (intermittently) due to thread leak - since scalatest runs all tests in same vm. Author: Mridul Muralidharan Closes #504 from mridulm/resource_leak_fixes and squashes the following commits: a5d10d0 [Mridul Muralidharan] Prevent thread leaks while running tests : cleanup all threads when SparkContext.stop is invoked. Causes tests to fail 7b5e19c [Mridul Muralidharan] Prevent NPE while running tests --- .../apache/spark/metrics/MetricsSystem.scala | 22 ++++++++------- .../spark/scheduler/TaskSchedulerImpl.scala | 1 + .../apache/spark/storage/BlockManager.scala | 2 ++ .../spark/storage/DiskBlockManager.scala | 28 +++++++++++-------- .../spark/storage/ShuffleBlockManager.scala | 4 +++ .../org/apache/spark/ui/JettyUtils.scala | 1 + .../spark/storage/DiskBlockManagerSuite.scala | 5 ++++ 7 files changed, 42 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index c5bda2078fc14..651511da1b7fe 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -129,17 +129,19 @@ private[spark] class MetricsSystem private (val instance: String, sinkConfigs.foreach { kv => val classPath = kv._2.getProperty("class") - try { - val sink = Class.forName(classPath) - .getConstructor(classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager]) - .newInstance(kv._2, registry, securityMgr) - if (kv._1 == "servlet") { - metricsServlet = Some(sink.asInstanceOf[MetricsServlet]) - } else { - sinks += sink.asInstanceOf[Sink] + if (null != classPath) { + try { + val sink = Class.forName(classPath) + .getConstructor(classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager]) + .newInstance(kv._2, registry, securityMgr) + if (kv._1 == "servlet") { + metricsServlet = Some(sink.asInstanceOf[MetricsServlet]) + } else { + sinks += sink.asInstanceOf[Sink] + } + } catch { + case e: Exception => logError("Sink class " + classPath + " cannot be instantialized", e) } - } catch { - case e: Exception => logError("Sink class " + classPath + " cannot be instantialized", e) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index be19d9b8854c8..5a68f38bc5844 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -356,6 +356,7 @@ private[spark] class TaskSchedulerImpl( if (taskResultGetter != null) { taskResultGetter.stop() } + starvationTimer.cancel() // sleeping for an arbitrary 1 seconds to ensure that messages are sent out. Thread.sleep(1000L) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index f15fa4dd7ffd5..ccd5c5320abe5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1021,6 +1021,8 @@ private[spark] class BlockManager( heartBeatTask.cancel() } connectionManager.stop() + shuffleBlockManager.stop() + diskBlockManager.stop() actorSystem.stop(slaveActor) blockInfo.clear() memoryStore.clear() diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 7a24c8f57f43b..054f66a8b7260 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -150,20 +150,26 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { override def run() { logDebug("Shutdown hook called") - localDirs.foreach { localDir => - try { - if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) - } catch { - case t: Throwable => - logError("Exception while deleting local spark dir: " + localDir, t) - } - } + stop() + } + }) + } - if (shuffleSender != null) { - shuffleSender.stop() + private[spark] def stop() { + localDirs.foreach { localDir => + if (localDir.isDirectory() && localDir.exists()) { + try { + if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) + } catch { + case t: Throwable => + logError("Exception while deleting local spark dir: " + localDir, t) } } - }) + } + + if (shuffleSender != null) { + shuffleSender.stop() + } } private[storage] def startShuffleBlockSender(port: Int): Int = { diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 4cd4cdbd9909d..35910e552fe86 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -207,6 +207,10 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { private def cleanup(cleanupTime: Long) { shuffleStates.clearOldValues(cleanupTime, (shuffleId, state) => removeShuffleBlocks(shuffleId)) } + + def stop() { + metadataCleaner.cancel() + } } private[spark] 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 750f5a501c213..fdeb15b5d058a 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -195,6 +195,7 @@ private[spark] object JettyUtils extends Logging { (server, server.getConnectors.head.getLocalPort) case f: Failure[_] => server.stop() + pool.stop() logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) logInfo("Error was: " + f.toString) connect((currentPort + 1) % 65536) diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 9b29e2a8a55de..42bfbf1bdfc74 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -53,6 +53,11 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { shuffleBlockManager.idToSegmentMap.clear() } + override def afterEach() { + diskBlockManager.stop() + shuffleBlockManager.idToSegmentMap.clear() + } + test("basic block creation") { val blockId = new TestBlockId("test") assertSegmentEquals(blockId, blockId.name, 0, 0) From 1fdf659d2fdf23c5562e5dc646d05083062281ed Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 24 Apr 2014 00:27:45 -0700 Subject: [PATCH 153/641] SPARK-1601 & SPARK-1602: two bug fixes related to cancellation This should go into 1.0 since it would return wrong data when the bug happens (which is pretty likely if cancellation is used). Test case attached. 1. Do not put partially executed partitions into cache (in task killing). 2. Iterator returned by CacheManager#getOrCompute was not an InterruptibleIterator, and was thus leading to uninterruptible jobs. Thanks @aarondav and @ahirreddy for reporting and helping debug. Author: Reynold Xin Closes #521 from rxin/kill and squashes the following commits: 401033f [Reynold Xin] Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/spark into kill 7a7bdd2 [Reynold Xin] Add a new line in the end of JobCancellationSuite.scala. 35cd9f7 [Reynold Xin] Fixed a bug that partially executed partitions can be put into cache (in task killing). --- .../scala/org/apache/spark/CacheManager.scala | 15 +++++-- .../apache/spark/InterruptibleIterator.scala | 12 +++++- .../apache/spark/TaskKilledException.scala | 23 ++++++++++ .../org/apache/spark/executor/Executor.scala | 8 ++-- .../apache/spark/JobCancellationSuite.scala | 43 ++++++++++++++++--- 5 files changed, 86 insertions(+), 15 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/TaskKilledException.scala diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index c7893f288b4b5..811610c657b62 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -47,7 +47,12 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { if (loading.contains(key)) { logInfo("Another thread is loading %s, waiting for it to finish...".format(key)) while (loading.contains(key)) { - try {loading.wait()} catch {case _ : Throwable =>} + try { + loading.wait() + } catch { + case e: Exception => + logWarning(s"Got an exception while waiting for another thread to load $key", e) + } } logInfo("Finished waiting for %s".format(key)) /* See whether someone else has successfully loaded it. The main way this would fail @@ -72,7 +77,9 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { val computedValues = rdd.computeOrReadCheckpoint(split, context) // Persist the result, so long as the task is not running locally - if (context.runningLocally) { return computedValues } + if (context.runningLocally) { + return computedValues + } // Keep track of blocks with updated statuses var updatedBlocks = Seq[(BlockId, BlockStatus)]() @@ -88,7 +95,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { updatedBlocks = blockManager.put(key, computedValues, storageLevel, tellMaster = true) blockManager.get(key) match { case Some(values) => - new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) + values.asInstanceOf[Iterator[T]] case None => logInfo("Failure to store %s".format(key)) throw new Exception("Block manager failed to return persisted valued") @@ -107,7 +114,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { val metrics = context.taskMetrics metrics.updatedBlocks = Some(updatedBlocks) - returnValue + new InterruptibleIterator(context, returnValue) } finally { loading.synchronized { diff --git a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala index fd1802ba2f984..ec11dbbffaaf8 100644 --- a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala +++ b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala @@ -24,7 +24,17 @@ package org.apache.spark private[spark] class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T]) extends Iterator[T] { - def hasNext: Boolean = !context.interrupted && delegate.hasNext + def hasNext: Boolean = { + // TODO(aarondav/rxin): Check Thread.interrupted instead of context.interrupted if interrupt + // is allowed. The assumption is that Thread.interrupted does not have a memory fence in read + // (just a volatile field in C), while context.interrupted is a volatile in the JVM, which + // introduces an expensive read fence. + if (context.interrupted) { + throw new TaskKilledException + } else { + delegate.hasNext + } + } def next(): T = delegate.next() } diff --git a/core/src/main/scala/org/apache/spark/TaskKilledException.scala b/core/src/main/scala/org/apache/spark/TaskKilledException.scala new file mode 100644 index 0000000000000..cbd6b2866e4f9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/TaskKilledException.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Exception for a task getting killed. + */ +private[spark] class TaskKilledException extends RuntimeException 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 914bc205cebe2..272bcda5f8f2f 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -161,8 +161,6 @@ private[spark] class Executor( class TaskRunner(execBackend: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) extends Runnable { - object TaskKilledException extends Exception - @volatile private var killed = false @volatile private var task: Task[Any] = _ @@ -200,7 +198,7 @@ private[spark] class Executor( // causes a NonLocalReturnControl exception to be thrown. The NonLocalReturnControl // exception will be caught by the catch block, leading to an incorrect ExceptionFailure // for the task. - throw TaskKilledException + throw new TaskKilledException } attemptedTask = Some(task) @@ -214,7 +212,7 @@ private[spark] class Executor( // If the task has been killed, let's fail it. if (task.killed) { - throw TaskKilledException + throw new TaskKilledException } val resultSer = SparkEnv.get.serializer.newInstance() @@ -257,7 +255,7 @@ private[spark] class Executor( execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) } - case TaskKilledException | _: InterruptedException if task.killed => { + case _: TaskKilledException | _: InterruptedException if task.killed => { logInfo("Executor killed task " + taskId) execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) } diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 16cfdf11c4a38..2c8ef405c944c 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -84,6 +84,35 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf assert(sc.parallelize(1 to 10, 2).count === 10) } + test("do not put partially executed partitions into cache") { + // In this test case, we create a scenario in which a partition is only partially executed, + // and make sure CacheManager does not put that partially executed partition into the + // BlockManager. + import JobCancellationSuite._ + sc = new SparkContext("local", "test") + + // Run from 1 to 10, and then block and wait for the task to be killed. + val rdd = sc.parallelize(1 to 1000, 2).map { x => + if (x > 10) { + taskStartedSemaphore.release() + taskCancelledSemaphore.acquire() + } + x + }.cache() + + val rdd1 = rdd.map(x => x) + + future { + taskStartedSemaphore.acquire() + sc.cancelAllJobs() + taskCancelledSemaphore.release(100000) + } + + intercept[SparkException] { rdd1.count() } + // If the partial block is put into cache, rdd.count() would return a number less than 1000. + assert(rdd.count() === 1000) + } + test("job group") { sc = new SparkContext("local[2]", "test") @@ -114,7 +143,6 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf assert(jobB.get() === 100) } - test("job group with interruption") { sc = new SparkContext("local[2]", "test") @@ -145,15 +173,14 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf assert(jobB.get() === 100) } -/* - test("two jobs sharing the same stage") { + ignore("two jobs sharing the same stage") { // sem1: make sure cancel is issued after some tasks are launched // sem2: make sure the first stage is not finished until cancel is issued val sem1 = new Semaphore(0) val sem2 = new Semaphore(0) sc = new SparkContext("local[2]", "test") - sc.dagScheduler.addSparkListener(new SparkListener { + sc.addSparkListener(new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { sem1.release() } @@ -179,7 +206,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf intercept[SparkException] { f1.get() } intercept[SparkException] { f2.get() } } - */ + def testCount() { // Cancel before launching any tasks { @@ -238,3 +265,9 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf } } } + + +object JobCancellationSuite { + val taskStartedSemaphore = new Semaphore(0) + val taskCancelledSemaphore = new Semaphore(0) +} From 27b2821cf16948962c7a6f513621a1eba60b8cf3 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 24 Apr 2014 09:57:28 -0700 Subject: [PATCH 154/641] [SPARK-1610] [SQL] Fix Cast to use exact type value when cast from BooleanType to NumericTy... ...pe. `Cast` from `BooleanType` to `NumericType` are all using `Int` value. But it causes `ClassCastException` when the casted value is used by the following evaluation like the code below: ``` scala scala> import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst._ scala> import types._ import types._ scala> import expressions._ import expressions._ scala> Add(Cast(Literal(true), ShortType), Literal(1.toShort)).eval() java.lang.ClassCastException: java.lang.Integer cannot be cast to java.lang.Short at scala.runtime.BoxesRunTime.unboxToShort(BoxesRunTime.java:102) at scala.math.Numeric$ShortIsIntegral$.plus(Numeric.scala:72) at org.apache.spark.sql.catalyst.expressions.Add$$anonfun$eval$2.apply(arithmetic.scala:58) at org.apache.spark.sql.catalyst.expressions.Add$$anonfun$eval$2.apply(arithmetic.scala:58) at org.apache.spark.sql.catalyst.expressions.Expression.n2(Expression.scala:114) at org.apache.spark.sql.catalyst.expressions.Add.eval(arithmetic.scala:58) at .(:17) at .() at .(:7) at .() at $print() at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:483) at scala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:734) at scala.tools.nsc.interpreter.IMain$Request.loadAndRun(IMain.scala:983) at scala.tools.nsc.interpreter.IMain.loadAndRunReq$1(IMain.scala:573) at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:604) at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:568) at scala.tools.nsc.interpreter.ILoop.reallyInterpret$1(ILoop.scala:760) at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:805) at scala.tools.nsc.interpreter.ILoop.command(ILoop.scala:717) at scala.tools.nsc.interpreter.ILoop.processLine$1(ILoop.scala:581) at scala.tools.nsc.interpreter.ILoop.innerLoop$1(ILoop.scala:588) at scala.tools.nsc.interpreter.ILoop.loop(ILoop.scala:591) at scala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply$mcZ$sp(ILoop.scala:882) at scala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply(ILoop.scala:837) at scala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply(ILoop.scala:837) at scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135) at scala.tools.nsc.interpreter.ILoop.process(ILoop.scala:837) at scala.tools.nsc.MainGenericRunner.runTarget$1(MainGenericRunner.scala:83) at scala.tools.nsc.MainGenericRunner.process(MainGenericRunner.scala:96) at scala.tools.nsc.MainGenericRunner$.main(MainGenericRunner.scala:105) at scala.tools.nsc.MainGenericRunner.main(MainGenericRunner.scala) ``` Author: Takuya UESHIN Closes #533 from ueshin/issues/SPARK-1610 and squashes the following commits: 70f36e8 [Takuya UESHIN] Fix Cast to use exact type value when cast from BooleanType to NumericType. --- .../apache/spark/sql/catalyst/expressions/Cast.scala | 10 +++++----- .../expressions/ExpressionEvaluationSuite.scala | 7 +++++++ 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 1f3fab09e9566..8b79b0cd65a84 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -111,7 +111,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => nullOrCast[String](_, s => try s.toLong catch { case _: NumberFormatException => null }) - case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1L else 0L) case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t)) case DecimalType => nullOrCast[BigDecimal](_, _.toLong) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toLong(b) @@ -131,7 +131,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => nullOrCast[String](_, s => try s.toShort catch { case _: NumberFormatException => null }) - case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1.toShort else 0.toShort) case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t).toShort) case DecimalType => nullOrCast[BigDecimal](_, _.toShort) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toShort @@ -141,7 +141,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => nullOrCast[String](_, s => try s.toByte catch { case _: NumberFormatException => null }) - case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1.toByte else 0.toByte) case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t).toByte) case DecimalType => nullOrCast[BigDecimal](_, _.toByte) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte @@ -162,7 +162,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => nullOrCast[String](_, s => try s.toDouble catch { case _: NumberFormatException => null }) - case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1d else 0d) case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t)) case DecimalType => nullOrCast[BigDecimal](_, _.toDouble) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toDouble(b) @@ -172,7 +172,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => nullOrCast[String](_, s => try s.toFloat catch { case _: NumberFormatException => null }) - case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1f else 0f) case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toFloat) case DecimalType => nullOrCast[BigDecimal](_, _.toFloat) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toFloat(b) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 2cd0d2b0e1385..4ce0dff9e1586 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -237,6 +237,13 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("2012-12-11" cast DoubleType, null) checkEvaluation(Literal(123) cast IntegerType, 123) + checkEvaluation(Literal(23d) + Cast(true, DoubleType), 24) + checkEvaluation(Literal(23) + Cast(true, IntegerType), 24) + checkEvaluation(Literal(23f) + Cast(true, FloatType), 24) + checkEvaluation(Literal(BigDecimal(23)) + Cast(true, DecimalType), 24) + checkEvaluation(Literal(23.toByte) + Cast(true, ByteType), 24) + checkEvaluation(Literal(23.toShort) + Cast(true, ShortType), 24) + intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)} } From faeb761cbe37521ec7ee6bbab7bcbc4bb1abb328 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 24 Apr 2014 09:59:44 -0700 Subject: [PATCH 155/641] Small changes to release script --- dev/create-release/create-release.sh | 4 +--- make-distribution.sh | 1 + 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index ad38c8d53eb2d..f1aa0f8f99e13 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -92,7 +92,6 @@ make_binary_release() { cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . rm -rf spark-$RELEASE_VERSION-bin-$NAME - tar cvzf spark-$RELEASE_VERSION-bin-$NAME.tgz spark-$RELEASE_VERSION-bin-$NAME echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour \ --output spark-$RELEASE_VERSION-bin-$NAME.tgz.asc \ --detach-sig spark-$RELEASE_VERSION-bin-$NAME.tgz @@ -102,7 +101,6 @@ make_binary_release() { echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \ SHA512 spark-$RELEASE_VERSION-bin-$NAME.tgz > \ spark-$RELEASE_VERSION-bin-$NAME.tgz.sha - rm -rf spark-$RELEASE_VERSION-bin-$NAME } make_binary_release "hadoop1" "--hadoop 1.0.4" @@ -114,7 +112,7 @@ echo "Copying release tarballs" ssh $USER_NAME@people.apache.org \ mkdir /home/$USER_NAME/public_html/spark-$RELEASE_VERSION-$RC_NAME rc_folder=spark-$RELEASE_VERSION-$RC_NAME -scp spark* \ +scp spark-* \ $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_folder/ # Docs diff --git a/make-distribution.sh b/make-distribution.sh index 83dfc745857ca..62a28217740cd 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -141,6 +141,7 @@ cp $FWDIR/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" # Copy other things mkdir "$DISTDIR"/conf cp "$FWDIR"/conf/*.template "$DISTDIR"/conf +cp "$FWDIR"/conf/slaves "$DISTDIR"/conf cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/python" "$DISTDIR" cp -r "$FWDIR/sbin" "$DISTDIR" From 6338a93f10e8b5f4365f71afd1086bdde64afad5 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 24 Apr 2014 10:06:18 -0700 Subject: [PATCH 156/641] SPARK-1488. Squash more language feature warnings in new commits by importing implicitConversion A recent commit reintroduced some of the same warnings that SPARK-1488 resolved. These are just a few more of the same changes to remove these warnings. Author: Sean Owen Closes #528 from srowen/SPARK-1488.2 and squashes the following commits: 62d592c [Sean Owen] More feature warnings in tests 4e2e94b [Sean Owen] Squash more language feature warnings in new commits by importing implicitConversion --- .../org/apache/spark/streaming/api/java/JavaInputDStream.scala | 1 + .../apache/spark/streaming/api/java/JavaPairInputDStream.scala | 2 ++ .../spark/streaming/api/java/JavaPairReceiverInputDStream.scala | 1 + .../spark/streaming/api/java/JavaReceiverInputDStream.scala | 1 + .../scala/org/apache/spark/streaming/NetworkReceiverSuite.scala | 1 + .../org/apache/spark/streaming/StreamingListenerSuite.scala | 1 + 6 files changed, 7 insertions(+) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala index 91f8d342d2bf9..ae5e693df93d9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala @@ -17,6 +17,7 @@ package org.apache.spark.streaming.api.java +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.streaming.dstream.InputDStream diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala index add858530862b..e6ff8a0cb545f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala @@ -18,6 +18,8 @@ package org.apache.spark.streaming.api.java import org.apache.spark.streaming.dstream.InputDStream + +import scala.language.implicitConversions import scala.reflect.ClassTag /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala index 974b3e451642d..294bc2be1898c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala @@ -17,6 +17,7 @@ package org.apache.spark.streaming.api.java +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.streaming.dstream.ReceiverInputDStream diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala index 340ef979808b9..8142dc59ea7c7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala @@ -17,6 +17,7 @@ package org.apache.spark.streaming.api.java +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.streaming.dstream.ReceiverInputDStream diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala index 5c0415ad14ebb..45304c76b0928 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer +import scala.language.postfixOps import org.apache.spark.SparkConf import org.apache.spark.storage.{StorageLevel, StreamBlockId} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 542c697ae3127..458dd3a2b13d8 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming import scala.collection.mutable.ArrayBuffer import scala.concurrent.Future import scala.concurrent.ExecutionContext.Implicits.global +import scala.language.postfixOps import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.DStream From 78a49b2532d4751257654dfe55a564bcd10701b3 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 24 Apr 2014 11:13:40 -0700 Subject: [PATCH 157/641] SPARK-1611: Fix incorrect initialization order in AppendOnlyMap JIRA: https://issues.apache.org/jira/browse/SPARK-1611 Author: zsxwing Closes #534 from zsxwing/SPARK-1611 and squashes the following commits: 96af089 [zsxwing] SPARK-1611: Fix incorrect initialization order in AppendOnlyMap --- .../org/apache/spark/util/collection/AppendOnlyMap.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index ad38250ad339f..1a6f1c2b55799 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -40,6 +40,8 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") require(initialCapacity >= 1, "Invalid initial capacity") + private val LOAD_FACTOR = 0.7 + private var capacity = nextPowerOf2(initialCapacity) private var mask = capacity - 1 private var curSize = 0 @@ -57,8 +59,6 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) private var destroyed = false private val destructionMessage = "Map state is invalid from destructive sorting!" - private val LOAD_FACTOR = 0.7 - /** Get the value for a given key */ def apply(key: K): V = { assert(!destroyed, destructionMessage) From bd375094a1480b0ff9c16ab8ddd2dba8731506df Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 24 Apr 2014 11:15:12 -0700 Subject: [PATCH 158/641] Spark 1490 Add kerberos support to the HistoryServer Here I've added the ability for the History server to login from a kerberos keytab file so that the history server can be run as a super user and stay up for along period of time while reading the history files from HDFS. Author: Thomas Graves Closes #513 from tgravescs/SPARK-1490 and squashes the following commits: e204a99 [Thomas Graves] remove extra logging 5418daa [Thomas Graves] fix typo in config 0076b99 [Thomas Graves] Update docs 4d76545 [Thomas Graves] SPARK-1490 Add kerberos support to the HistoryServer --- .../apache/spark/deploy/SparkHadoopUtil.scala | 4 ++++ .../spark/deploy/history/HistoryServer.scala | 16 +++++++++++++ docs/monitoring.md | 24 +++++++++++++++++++ 3 files changed, 44 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 9bdbfb33bf54f..498fcc520ac5e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -75,6 +75,10 @@ class SparkHadoopUtil { def getSecretKeyFromUserCredentials(key: String): Array[Byte] = { null } + def loginUserFromKeytab(principalName: String, keytabFilename: String) { + UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename) + } + } object SparkHadoopUtil { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index b8f56234d37ec..d7a3246bcfb47 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -22,6 +22,7 @@ import scala.collection.mutable import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler._ import org.apache.spark.ui.{WebUI, SparkUI} import org.apache.spark.ui.JettyUtils._ @@ -257,6 +258,7 @@ object HistoryServer { val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR def main(argStrings: Array[String]) { + initSecurity() val args = new HistoryServerArguments(argStrings) val securityManager = new SecurityManager(conf) val server = new HistoryServer(args.logDir, securityManager, conf) @@ -266,6 +268,20 @@ object HistoryServer { while(true) { Thread.sleep(Int.MaxValue) } server.stop() } + + def initSecurity() { + // If we are accessing HDFS and it has security enabled (Kerberos), we have to login + // from a keytab file so that we can access HDFS beyond the kerberos ticket expiration. + // As long as it is using Hadoop rpc (hdfs://), a relogin will automatically + // occur from the keytab. + if (conf.getBoolean("spark.history.kerberos.enabled", false)) { + // if you have enabled kerberos the following 2 params must be set + val principalName = conf.get("spark.history.kerberos.principal") + val keytabFilename = conf.get("spark.history.kerberos.keytab") + SparkHadoopUtil.get.loginUserFromKeytab(principalName, keytabFilename) + } + } + } diff --git a/docs/monitoring.md b/docs/monitoring.md index 144be3daf1208..347a9b1f1a329 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -91,6 +91,30 @@ represents an application's event logs. This creates a web interface at The port to which the web interface of the history server binds. + + + + + + + + + + + + + + +
      Property NameDefaultMeaning
      spark.history.kerberos.enabledfalse + Indicates whether the history server should use kerberos to login. This is useful + if the history server is accessing HDFS files on a secure Hadoop cluster. If this is + true it looks uses the configs spark.history.kerberos.principal and + spark.history.kerberos.keytab. +
      spark.history.kerberos.principal(none) + Kerberos principal name for the History Server. +
      spark.history.kerberos.keytab(none) + Location of the kerberos keytab file for the History Server. +
      Note that in all of these UIs, the tables are sortable by clicking their headers, From c5c1916dd1b77e22759d58b5b361c56672983e3e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 24 Apr 2014 14:54:01 -0700 Subject: [PATCH 159/641] SPARK-1494 Don't initialize classes loaded by MIMA excludes, attempt 2 [WIP] Looks like scala reflection was invoking the static initializer: ``` ... at org.apache.spark.sql.test.TestSQLContext$.(TestSQLContext.scala:25) at org.apache.spark.sql.test.TestSQLContext$.(TestSQLContext.scala) at java.lang.Class.forName0(Native Method) at java.lang.Class.forName(Class.java:270) at scala.reflect.runtime.JavaMirrors$JavaMirror.javaClass(JavaMirrors.scala:500) at scala.reflect.runtime.JavaMirrors$JavaMirror.tryJavaClass(JavaMirrors.scala:505) at scala.reflect.runtime.SymbolLoaders$PackageScope.lookupEntry(SymbolLoaders.scala:109) ... ``` Need to make sure that this doesn't change the exclusion semantics before merging. Author: Michael Armbrust Closes #526 from marmbrus/mima and squashes the following commits: 8168dea [Michael Armbrust] Spurious change afba262 [Michael Armbrust] Prevent Scala reflection from running static class initializer. --- .../main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala index d9978b02a9f3e..a433e8e2e89f4 100644 --- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala +++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala @@ -50,7 +50,7 @@ object GenerateMIMAIgnore { is a module or class. */ val privateAsClass = mirror - .staticClass(className) + .classSymbol(Class.forName(className, false, classLoader)) .privateWithin .fullName .startsWith(packageName) From a03ac222d84025a1036750e1179136a13f75dea7 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Thu, 24 Apr 2014 15:07:23 -0700 Subject: [PATCH 160/641] Fix Scala Style Any comments are welcome Author: Sandeep Closes #531 from techaddict/stylefix-1 and squashes the following commits: 7492730 [Sandeep] Pass 4 98b2428 [Sandeep] fix rxin suggestions b5e2e6f [Sandeep] Pass 3 05932d7 [Sandeep] fix if else styling 2 08690e5 [Sandeep] fix if else styling --- .../scala/org/apache/spark/Accumulators.scala | 7 +- .../spark/deploy/SparkSubmitArguments.scala | 3 +- .../apache/spark/deploy/master/Master.scala | 3 +- .../spark/deploy/worker/DriverRunner.scala | 8 ++- .../spark/deploy/worker/ui/LogPage.scala | 16 +++-- .../apache/spark/storage/BlockManager.scala | 8 ++- .../spark/util/BoundedPriorityQueue.scala | 12 ++-- .../org/apache/spark/util/FileLogger.scala | 4 +- .../scala/org/apache/spark/util/Utils.scala | 3 +- .../scala/org/apache/spark/rdd/RDDSuite.scala | 4 +- .../org/apache/spark/examples/LogQuery.scala | 3 +- .../examples/clickstream/PageViewStream.scala | 7 +- .../apache/spark/graphx/GraphOpsSuite.scala | 7 +- .../apache/spark/repl/SparkExprTyper.scala | 13 ++-- .../sql/parquet/ParquetTableOperations.scala | 5 +- .../sql/parquet/ParquetTableSupport.scala | 7 +- .../streaming/scheduler/ReceiverTracker.scala | 3 +- .../spark/deploy/yarn/ApplicationMaster.scala | 9 +-- .../spark/deploy/yarn/ExecutorLauncher.scala | 3 +- .../deploy/yarn/YarnAllocationHandler.scala | 67 ++++++++++--------- 20 files changed, 109 insertions(+), 83 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index d5f3e3f6ec496..6d652faae149a 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -104,8 +104,11 @@ class Accumulable[R, T] ( * Set the accumulator's value; only allowed on master. */ def value_= (newValue: R) { - if (!deserialized) value_ = newValue - else throw new UnsupportedOperationException("Can't assign accumulator value in task") + if (!deserialized) { + value_ = newValue + } else { + throw new UnsupportedOperationException("Can't assign accumulator value in task") + } } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index cc976565cc72f..c3e8c6b8c65a9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -66,8 +66,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { if (k.startsWith("spark")) { defaultProperties(k) = v if (verbose) SparkSubmit.printStream.println(s"Adding default property: $k=$v") - } - else { + } else { SparkSubmit.printWarning(s"Ignoring non-spark config property: $k=$v") } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 81f990bfa6513..fdb633bd33608 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -237,8 +237,7 @@ private[spark] class Master( if (waitingDrivers.contains(d)) { waitingDrivers -= d self ! DriverStateChanged(driverId, DriverState.KILLED, None) - } - else { + } else { // We just notify the worker to kill the driver here. The final bookkeeping occurs // on the return path when the worker submits a state change back to the master // to notify it that the driver was successfully killed. diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index f918b42c83bc6..662d37871e7a6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -91,9 +91,11 @@ private[spark] class DriverRunner( } val state = - if (killed) { DriverState.KILLED } - else if (finalException.isDefined) { DriverState.ERROR } - else { + if (killed) { + DriverState.KILLED + } else if (finalException.isDefined) { + DriverState.ERROR + } else { finalExitCode match { case Some(0) => DriverState.FINISHED case _ => DriverState.FAILED diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index fec1207948628..8381f59672ea3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -89,8 +89,7 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { Previous {Utils.bytesToString(math.min(byteLength, startByte))} - } - else { + } else { @@ -104,8 +103,7 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { Next {Utils.bytesToString(math.min(byteLength, logLength - endByte))} - } - else { + } else { @@ -137,9 +135,13 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { val logLength = file.length() val getOffset = offset.getOrElse(logLength - defaultBytes) val startByte = - if (getOffset < 0) 0L - else if (getOffset > logLength) logLength - else getOffset + if (getOffset < 0) { + 0L + } else if (getOffset > logLength) { + logLength + } else { + getOffset + } val logPageLength = math.min(byteLength, maxBytes) val endByte = math.min(startByte + logPageLength, logLength) (startByte, endByte) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index ccd5c5320abe5..02ba5ecf52459 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -281,7 +281,9 @@ private[spark] class BlockManager( val onDiskSize = status.diskSize master.updateBlockInfo( blockManagerId, blockId, storageLevel, inMemSize, onDiskSize, inTachyonSize) - } else true + } else { + true + } } /** @@ -676,7 +678,7 @@ private[spark] class BlockManager( tachyonStore.putValues(blockId, iterator, level, false) case ArrayBufferValues(array) => tachyonStore.putValues(blockId, array, level, false) - case ByteBufferValues(bytes) => + case ByteBufferValues(bytes) => bytes.rewind() tachyonStore.putBytes(blockId, bytes, level) } @@ -695,7 +697,7 @@ private[spark] class BlockManager( diskStore.putValues(blockId, iterator, level, askForBytes) case ArrayBufferValues(array) => diskStore.putValues(blockId, array, level, askForBytes) - case ByteBufferValues(bytes) => + case ByteBufferValues(bytes) => bytes.rewind() diskStore.putBytes(blockId, bytes, level) } diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index b9f4a5d720b93..1b2b1932e0c3d 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -43,8 +43,11 @@ private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Orderin } override def +=(elem: A): this.type = { - if (size < maxSize) underlying.offer(elem) - else maybeReplaceLowest(elem) + if (size < maxSize) { + underlying.offer(elem) + } else { + maybeReplaceLowest(elem) + } this } @@ -59,7 +62,8 @@ private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Orderin if (head != null && ord.gt(a, head)) { underlying.poll() underlying.offer(a) - } else false + } else { + false + } } } - diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 7d47b2a72aff7..1ed3b70bb24fd 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -113,7 +113,9 @@ private[spark] class FileLogger( * @param withTime Whether to prepend message with a timestamp */ def log(msg: String, withTime: Boolean = false) { - val writeInfo = if (!withTime) msg else { + val writeInfo = if (!withTime) { + msg + } else { val date = new Date(System.currentTimeMillis()) dateFormat.get.format(date) + ": " + msg } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index a3af4e7b91692..d333e2a88c18c 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -811,8 +811,7 @@ private[spark] object Utils extends Logging { } else { el.getMethodName } - } - else { + } else { firstUserLine = el.getLineNumber firstUserFile = el.getFileName firstUserClass = el.getClassName 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 d7c90346d88a2..2676558bfc86d 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -381,8 +381,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { val prng42 = new Random(42) val prng43 = new Random(43) Array(1, 2, 3, 4, 5, 6).filter{i => - if (i < 4) 0 == prng42.nextInt(3) - else 0 == prng43.nextInt(3)} + if (i < 4) 0 == prng42.nextInt(3) else 0 == prng43.nextInt(3) + } } assert(sample.size === checkSample.size) for (i <- 0 until sample.size) assert(sample(i) === checkSample(i)) diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index 820e87d04f3fa..f77a444ff7a9f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -49,8 +49,7 @@ object LogQuery { System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val dataSet = - if (args.length == 2) sc.textFile(args(1)) - else sc.parallelize(exampleApacheLogs) + if (args.length == 2) sc.textFile(args(1)) else sc.parallelize(exampleApacheLogs) // scalastyle:off val apacheLogRegex = """^([\d.]+) (\S+) (\S+) \[([\w\d:/]+\s[+\-]\d{4})\] "(.+?)" (\d{3}) ([\d\-]+) "([^"]+)" "([^"]+)".*""".r diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala index edc769c59734a..673013f7cf948 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala @@ -69,8 +69,11 @@ object PageViewStream { val normalCount = statuses.filter(_ == 200).size val errorCount = statuses.size - normalCount val errorRatio = errorCount.toFloat / statuses.size - if (errorRatio > 0.05) {"%s: **%s**".format(zip, errorRatio)} - else {"%s: %s".format(zip, errorRatio)} + if (errorRatio > 0.05) { + "%s: **%s**".format(zip, errorRatio) + } else { + "%s: %s".format(zip, errorRatio) + } } // Return the number unique users in last 15 seconds diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala index a467ca1ae715a..ea94d4accb63b 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -165,8 +165,11 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { // not have any edges in the specified direction. assert(edges.count === 50) edges.collect.foreach { - case (vid, edges) => if (vid > 0 && vid < 49) assert(edges.size == 2) - else assert(edges.size == 1) + case (vid, edges) => if (vid > 0 && vid < 49) { + assert(edges.size == 2) + } else { + assert(edges.size == 1) + } } edges.collect.foreach { case (vid, edges) => diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala index dcc139544e2c2..f8432c8af6ed2 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala @@ -47,9 +47,13 @@ trait SparkExprTyper extends Logging { var isIncomplete = false reporter.withIncompleteHandler((_, _) => isIncomplete = true) { val trees = codeParser.stmts(line) - if (reporter.hasErrors) Some(Nil) - else if (isIncomplete) None - else Some(trees) + if (reporter.hasErrors) { + Some(Nil) + } else if (isIncomplete) { + None + } else { + Some(trees) + } } } // def parsesAsExpr(line: String) = { @@ -70,8 +74,7 @@ trait SparkExprTyper extends Logging { val sym0 = symbolOfTerm(name) // drop NullaryMethodType val sym = sym0.cloneSymbol setInfo afterTyper(sym0.info.finalResultType) - if (sym.info.typeSymbol eq UnitClass) NoSymbol - else sym + if (sym.info.typeSymbol eq UnitClass) NoSymbol else sym case _ => NoSymbol } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index d5846baa72ada..f825ca3c028ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -203,8 +203,9 @@ case class InsertIntoParquetTable( val stageId = sc.newRddId() val taskIdOffset = - if (overwrite) 1 - else { + if (overwrite) { + 1 + } else { FileSystemHelper .findMaxTaskId(NewFileOutputFormat.getOutputPath(job).toString, job.getConfiguration) + 1 } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 84b1b4609458b..71ba0fecce47a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -158,8 +158,11 @@ private[parquet] class CatalystGroupConverter( a => a.dataType match { case ctype: NativeType => // note: for some reason matching for StringType fails so use this ugly if instead - if (ctype == StringType) new CatalystPrimitiveStringConverter(this, schema.indexOf(a)) - else new CatalystPrimitiveConverter(this, schema.indexOf(a)) + if (ctype == StringType) { + new CatalystPrimitiveStringConverter(this, schema.indexOf(a)) + } else { + new CatalystPrimitiveConverter(this, schema.indexOf(a)) + } case _ => throw new RuntimeException( s"unable to convert datatype ${a.dataType.toString} in CatalystGroupConverter") } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 3d2537f6f23dc..557e0961d5944 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -240,8 +240,7 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { if (hasLocationPreferences) { val receiversWithPreferences = receivers.map(r => (r, Seq(r.preferredLocation.get))) ssc.sc.makeRDD[Receiver[_]](receiversWithPreferences) - } - else { + } else { ssc.sc.makeRDD(receivers, receivers.size) } 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 2f74965900baf..fc13dbecb4555 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 @@ -147,12 +147,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) .orElse(Option(System.getenv("LOCAL_DIRS"))) - + localDirs match { case None => throw new Exception("Yarn Local dirs can't be empty") case Some(l) => l } - } + } private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() @@ -321,8 +321,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, logInfo("Allocating %d containers to make up for (potentially) lost containers". format(missingExecutorCount)) yarnAllocator.allocateContainers(missingExecutorCount) + } else { + sendProgress() } - else sendProgress() Thread.sleep(sleepTime) } } @@ -361,7 +362,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, return } isFinished = true - + logInfo("finishApplicationMaster with " + status) if (registered) { val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index ea356f33eb998..65b7215afbd4c 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -243,8 +243,9 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp logInfo("Allocating " + missingExecutorCount + " containers to make up for (potentially ?) lost containers") yarnAllocator.allocateContainers(missingExecutorCount) + } else { + sendProgress() } - else sendProgress() Thread.sleep(sleepTime) } } 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 95f0f9d0ff2bc..856391e52b2df 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 @@ -60,12 +60,12 @@ object AllocationType extends Enumeration { */ private[yarn] class YarnAllocationHandler( val conf: Configuration, - val resourceManager: AMRMProtocol, + val resourceManager: AMRMProtocol, val appAttemptId: ApplicationAttemptId, val maxExecutors: Int, val executorMemory: Int, val executorCores: Int, - val preferredHostToCount: Map[String, Int], + val preferredHostToCount: Map[String, Int], val preferredRackToCount: Map[String, Int], val sparkConf: SparkConf) extends Logging { @@ -136,9 +136,10 @@ private[yarn] class YarnAllocationHandler( val containers = hostToContainers.getOrElseUpdate(host, new ArrayBuffer[Container]()) containers += container + } else { + // Add all ignored containers to released list + releasedContainerList.add(container.getId()) } - // Add all ignored containers to released list - else releasedContainerList.add(container.getId()) } // Find the appropriate containers to use. Slightly non trivial groupBy ... @@ -159,8 +160,7 @@ private[yarn] class YarnAllocationHandler( dataLocalContainers.put(candidateHost, remainingContainers) // all consumed remainingContainers = null - } - else if (requiredHostCount > 0) { + } else if (requiredHostCount > 0) { // Container list has more containers than we need for data locality. // Split into two : data local container count of (remainingContainers.size - // requiredHostCount) and rest as remainingContainer @@ -170,7 +170,7 @@ private[yarn] class YarnAllocationHandler( // remainingContainers = remaining // yarn has nasty habit of allocating a tonne of containers on a host - discourage this : - // add remaining to release list. If we have insufficient containers, next allocation + // add remaining to release list. If we have insufficient containers, next allocation // cycle will reallocate (but wont treat it as data local) for (container <- remaining) releasedContainerList.add(container.getId()) remainingContainers = null @@ -182,7 +182,7 @@ private[yarn] class YarnAllocationHandler( if (rack != null){ val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0) - val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - + val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - rackLocalContainers.get(rack).getOrElse(List()).size @@ -191,8 +191,7 @@ private[yarn] class YarnAllocationHandler( dataLocalContainers.put(rack, remainingContainers) // All consumed remainingContainers = null - } - else if (requiredRackCount > 0) { + } else if (requiredRackCount > 0) { // container list has more containers than we need for data locality. // Split into two : data local container count of (remainingContainers.size - // requiredRackCount) and rest as remainingContainer @@ -213,7 +212,7 @@ private[yarn] class YarnAllocationHandler( } } - // Now that we have split the containers into various groups, go through them in order : + // Now that we have split the containers into various groups, go through them in order : // first host local, then rack local and then off rack (everything else). // Note that the list we create below tries to ensure that not all containers end up within a // host if there are sufficiently large number of hosts/containers. @@ -238,8 +237,7 @@ private[yarn] class YarnAllocationHandler( releasedContainerList.add(containerId) // reset counter back to old value. numExecutorsRunning.decrementAndGet() - } - else { + } else { // Deallocate + allocate can result in reusing id's wrongly - so use a different counter // (executorIdCounter) val executorId = executorIdCounter.incrementAndGet().toString @@ -293,8 +291,7 @@ private[yarn] class YarnAllocationHandler( // Was this released by us ? If yes, then simply remove from containerSet and move on. if (pendingReleaseContainers.containsKey(containerId)) { pendingReleaseContainers.remove(containerId) - } - else { + } else { // Simply decrement count - next iteration of ReporterThread will take care of allocating. numExecutorsRunning.decrementAndGet() logInfo("Completed container %s (state: %s, exit status: %s)".format( @@ -319,8 +316,11 @@ private[yarn] class YarnAllocationHandler( assert (containerSet != null) containerSet -= containerId - if (containerSet.isEmpty) allocatedHostToContainersMap.remove(host) - else allocatedHostToContainersMap.update(host, containerSet) + if (containerSet.isEmpty) { + allocatedHostToContainersMap.remove(host) + } else { + allocatedHostToContainersMap.update(host, containerSet) + } allocatedContainerToHostMap -= containerId @@ -328,8 +328,11 @@ private[yarn] class YarnAllocationHandler( val rack = YarnAllocationHandler.lookupRack(conf, host) if (rack != null) { val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 - if (rackCount > 0) allocatedRackCount.put(rack, rackCount) - else allocatedRackCount.remove(rack) + if (rackCount > 0) { + allocatedRackCount.put(rack, rackCount) + } else { + allocatedRackCount.remove(rack) + } } } } @@ -365,10 +368,10 @@ private[yarn] class YarnAllocationHandler( } } - val requestedContainers: ArrayBuffer[ResourceRequest] = + val requestedContainers: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest](rackToCounts.size) for ((rack, count) <- rackToCounts){ - requestedContainers += + requestedContainers += createResourceRequest(AllocationType.RACK, rack, count, YarnAllocationHandler.PRIORITY) } @@ -401,11 +404,10 @@ private[yarn] class YarnAllocationHandler( preferredHostToCount.isEmpty) resourceRequests = List(createResourceRequest( AllocationType.ANY, null, numExecutors, YarnAllocationHandler.PRIORITY)) - } - else { - // request for all hosts in preferred nodes and for numExecutors - + } else { + // request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. - val hostContainerRequests: ArrayBuffer[ResourceRequest] = + val hostContainerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest](preferredHostToCount.size) for ((candidateHost, candidateCount) <- preferredHostToCount) { val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) @@ -449,8 +451,7 @@ private[yarn] class YarnAllocationHandler( if (numExecutors > 0) { logInfo("Allocating %d executor containers with %d of memory each.".format(numExecutors, executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) - } - else { + } else { logDebug("Empty allocation req .. release : " + releasedContainerList) } @@ -467,7 +468,7 @@ private[yarn] class YarnAllocationHandler( private def createResourceRequest( - requestType: AllocationType.AllocationType, + requestType: AllocationType.AllocationType, resource:String, numExecutors: Int, priority: Int): ResourceRequest = { @@ -528,7 +529,7 @@ private[yarn] class YarnAllocationHandler( if (! retval.isEmpty) { releasedContainerList.removeAll(retval) for (v <- retval) pendingReleaseContainers.put(v, true) - logInfo("Releasing " + retval.size + " containers. pendingReleaseContainers : " + + logInfo("Releasing " + retval.size + " containers. pendingReleaseContainers : " + pendingReleaseContainers) } @@ -539,7 +540,7 @@ private[yarn] class YarnAllocationHandler( object YarnAllocationHandler { val ANY_HOST = "*" - // All requests are issued with same priority : we do not (yet) have any distinction between + // All requests are issued with same priority : we do not (yet) have any distinction between // request types (like map/reduce in hadoop for example) val PRIORITY = 1 @@ -548,7 +549,7 @@ object YarnAllocationHandler { // Host to rack map - saved from allocation requests // We are expecting this not to change. - // Note that it is possible for this to change : and RM will indicate that to us via update + // Note that it is possible for this to change : and RM will indicate that to us via update // response to allocate. But we are punting on handling that for now. private val hostToRack = new ConcurrentHashMap[String, String]() private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]() @@ -565,7 +566,7 @@ object YarnAllocationHandler { conf, resourceManager, appAttemptId, - args.numExecutors, + args.numExecutors, args.executorMemory, args.executorCores, Map[String, Int](), @@ -587,7 +588,7 @@ object YarnAllocationHandler { conf, resourceManager, appAttemptId, - args.numExecutors, + args.numExecutors, args.executorMemory, args.executorCores, hostToCount, From f99af8529b6969986f0c3e03f6ff9b7bb9d53ece Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 24 Apr 2014 15:55:18 -0700 Subject: [PATCH 161/641] SPARK-1104: kill Process in workerThread of ExecutorRunner As reported in https://spark-project.atlassian.net/browse/SPARK-1104 By @pwendell: "Sometimes due to large shuffles executors will take a long time shutting down. In particular this can happen if large numbers of shuffle files are around (this will be alleviated by SPARK-1103, but nonetheless...). The symptom is you have DEAD workers sitting around in the UI and the existing workers keep trying to re-register but can't because they've been assumed dead." In this patch, I add lines in the handler of InterruptedException in workerThread of executorRunner, so that the process.destroy() and process.waitFor() can only block the workerThread instead of blocking the worker Actor... --------- analysis: process.destroy() is a blocking method, i.e. it only returns when all shutdownHook threads return...so calling it in Worker thread will make Worker block for a long while.... about what will happen on the shutdown hooks when the JVM process is killed: http://www.tutorialspoint.com/java/lang/runtime_addshutdownhook.htm Author: CodingCat Closes #35 from CodingCat/SPARK-1104 and squashes the following commits: 85767da [CodingCat] add null checking and remove unnecessary killProce 3107aeb [CodingCat] address Aaron's comments eb615ba [CodingCat] kill the process when the error happens 0accf2f [CodingCat] set process to null after killed it 1d511c8 [CodingCat] kill Process in workerThread --- .../spark/deploy/worker/ExecutorRunner.scala | 31 +++++++++---------- 1 file changed, 14 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index f94cd685e8eb0..2051403682737 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -58,30 +58,29 @@ private[spark] class ExecutorRunner( override def run() { fetchAndRunExecutor() } } workerThread.start() - // Shutdown hook that kills actors on shutdown. shutdownHook = new Thread() { override def run() { - if (process != null) { - logInfo("Shutdown hook killing child process.") - process.destroy() - process.waitFor() - } + killProcess() } } Runtime.getRuntime.addShutdownHook(shutdownHook) } + private def killProcess() { + if (process != null) { + logInfo("Killing process!") + process.destroy() + process.waitFor() + } + } + /** Stop this executor runner, including killing the process it launched */ def kill() { if (workerThread != null) { + // the workerThread will kill the child process when interrupted workerThread.interrupt() workerThread = null - if (process != null) { - logInfo("Killing process!") - process.destroy() - process.waitFor() - } state = ExecutorState.KILLED worker ! ExecutorStateChanged(appId, execId, state, None, None) Runtime.getRuntime.removeShutdownHook(shutdownHook) @@ -128,7 +127,6 @@ private[spark] class ExecutorRunner( // parent process for the executor command env.put("SPARK_LAUNCH_WITH_SCALA", "0") process = builder.start() - val header = "Spark Executor Command: %s\n%s\n\n".format( command.mkString("\"", "\" \"", "\""), "=" * 40) @@ -148,14 +146,13 @@ private[spark] class ExecutorRunner( val message = "Command exited with code " + exitCode worker ! ExecutorStateChanged(appId, execId, state, Some(message), Some(exitCode)) } catch { - case interrupted: InterruptedException => + case interrupted: InterruptedException => { logInfo("Runner thread for executor " + fullId + " interrupted") - + killProcess() + } case e: Exception => { logError("Error running executor", e) - if (process != null) { - process.destroy() - } + killProcess() state = ExecutorState.FAILED val message = e.getClass + ": " + e.getMessage worker ! ExecutorStateChanged(appId, execId, state, Some(message), None) From 35e3d199f04fba3230625002a458d43b9578b2e8 Mon Sep 17 00:00:00 2001 From: Arun Ramakrishnan Date: Thu, 24 Apr 2014 17:27:16 -0700 Subject: [PATCH 162/641] SPARK-1438 RDD.sample() make seed param optional copying form previous pull request https://github.com/apache/spark/pull/462 Its probably better to let the underlying language implementation take care of the default . This was easier to do with python as the default value for seed in random and numpy random is None. In Scala/Java side it might mean propagating an Option or null(oh no!) down the chain until where the Random is constructed. But, looks like the convention in some other methods was to use System.nanoTime. So, followed that convention. Conflict with overloaded method in sql.SchemaRDD.sample which also defines default params. sample(fraction, withReplacement=false, seed=math.random) Scala does not allow more than one overloaded to have default params. I believe the author intended to override the RDD.sample method and not overload it. So, changed it. If backward compatible is important, 3 new method can be introduced (without default params) like this sample(fraction) sample(fraction, withReplacement) sample(fraction, withReplacement, seed) Added some tests for the scala RDD takeSample method. Author: Arun Ramakrishnan This patch had conflicts when merged, resolved by Committer: Matei Zaharia Closes #477 from smartnut007/master and squashes the following commits: 07bb06e [Arun Ramakrishnan] SPARK-1438 fixing more space formatting issues b9ebfe2 [Arun Ramakrishnan] SPARK-1438 removing redundant import of random in python rddsampler 8d05b1a [Arun Ramakrishnan] SPARK-1438 RDD . Replace System.nanoTime with a Random generated number. python: use a separate instance of Random instead of seeding language api global Random instance. 69619c6 [Arun Ramakrishnan] SPARK-1438 fix spacing issue 0c247db [Arun Ramakrishnan] SPARK-1438 RDD language apis to support optional seed in RDD methods sample/takeSample --- .../apache/spark/api/java/JavaDoubleRDD.scala | 9 +++++- .../apache/spark/api/java/JavaPairRDD.scala | 9 +++++- .../org/apache/spark/api/java/JavaRDD.scala | 9 +++++- .../apache/spark/api/java/JavaRDDLike.scala | 6 +++- .../spark/rdd/PartitionwiseSampledRDD.scala | 5 +-- .../main/scala/org/apache/spark/rdd/RDD.scala | 11 ++++--- .../scala/org/apache/spark/util/Utils.scala | 2 ++ .../scala/org/apache/spark/rdd/RDDSuite.scala | 21 ++++++++++++- python/pyspark/rdd.py | 13 ++++---- python/pyspark/rddsampler.py | 31 +++++++++---------- .../plans/logical/basicOperators.scala | 2 +- .../org/apache/spark/sql/SchemaRDD.scala | 5 +-- .../spark/sql/execution/basicOperators.scala | 6 ++-- 13 files changed, 88 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index 4330cef3965ee..a6123bd108c11 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -30,6 +30,7 @@ import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.util.StatCounter +import org.apache.spark.util.Utils class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, JavaDoubleRDD] { @@ -133,7 +134,13 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, Ja /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: JDouble, seed: Int): JavaDoubleRDD = + def sample(withReplacement: Boolean, fraction: JDouble): JavaDoubleRDD = + sample(withReplacement, fraction, Utils.random.nextLong) + + /** + * Return a sampled subset of this RDD. + */ + def sample(withReplacement: Boolean, fraction: JDouble, seed: Long): JavaDoubleRDD = fromRDD(srdd.sample(withReplacement, fraction, seed)) /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index b3ec270281ae4..554c065358648 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -39,6 +39,7 @@ import org.apache.spark.api.java.function.{Function => JFunction, Function2 => J import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.{OrderedRDDFunctions, RDD} import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) (implicit val kClassTag: ClassTag[K], implicit val vClassTag: ClassTag[V]) @@ -119,7 +120,13 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaPairRDD[K, V] = + def sample(withReplacement: Boolean, fraction: Double): JavaPairRDD[K, V] = + sample(withReplacement, fraction, Utils.random.nextLong) + + /** + * Return a sampled subset of this RDD. + */ + def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.sample(withReplacement, fraction, seed)) /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 327c1552dc941..dc698dea75e43 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -24,6 +24,7 @@ import org.apache.spark._ import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) extends JavaRDDLike[T, JavaRDD[T]] { @@ -98,7 +99,13 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaRDD[T] = + def sample(withReplacement: Boolean, fraction: Double): JavaRDD[T] = + sample(withReplacement, fraction, Utils.random.nextLong) + + /** + * Return a sampled subset of this RDD. + */ + def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaRDD[T] = wrapRDD(rdd.sample(withReplacement, fraction, seed)) /** 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 725c423a53e35..574a98636a619 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 @@ -34,6 +34,7 @@ import org.apache.spark.api.java.function.{Function => JFunction, Function2 => J import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def wrapRDD(rdd: RDD[T]): This @@ -394,7 +395,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { new java.util.ArrayList(arr) } - def takeSample(withReplacement: Boolean, num: Int, seed: Int): JList[T] = { + def takeSample(withReplacement: Boolean, num: Int): JList[T] = + takeSample(withReplacement, num, Utils.random.nextLong) + + def takeSample(withReplacement: Boolean, num: Int, seed: Long): JList[T] = { import scala.collection.JavaConversions._ val arr: java.util.Collection[T] = rdd.takeSample(withReplacement, num, seed).toSeq new java.util.ArrayList(arr) diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala index b4e3bb5d75e17..b5b8a5706deb3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala @@ -23,6 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.{Partition, TaskContext} import org.apache.spark.util.random.RandomSampler +import org.apache.spark.util.Utils private[spark] class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) @@ -38,14 +39,14 @@ class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) * * @param prev RDD to be sampled * @param sampler a random sampler - * @param seed random seed, default to System.nanoTime + * @param seed random seed * @tparam T input RDD item type * @tparam U sampled RDD item type */ private[spark] class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( prev: RDD[T], sampler: RandomSampler[T, U], - @transient seed: Long = System.nanoTime) + @transient seed: Long = Utils.random.nextLong) extends RDD[U](prev) { override def getPartitions: Array[Partition] = { 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 6c897cc03b641..e8bbfbf01679c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -341,7 +341,9 @@ abstract class RDD[T: ClassTag]( /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: Double, seed: Int): RDD[T] = { + def sample(withReplacement: Boolean, + fraction: Double, + seed: Long = Utils.random.nextLong): RDD[T] = { require(fraction >= 0.0, "Invalid fraction value: " + fraction) if (withReplacement) { new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), seed) @@ -354,11 +356,11 @@ abstract class RDD[T: ClassTag]( * Randomly splits this RDD with the provided weights. * * @param weights weights for splits, will be normalized if they don't sum to 1 - * @param seed random seed, default to System.nanoTime + * @param seed random seed * * @return split RDDs in an array */ - def randomSplit(weights: Array[Double], seed: Long = System.nanoTime): Array[RDD[T]] = { + def randomSplit(weights: Array[Double], seed: Long = Utils.random.nextLong): Array[RDD[T]] = { val sum = weights.sum val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) normalizedCumWeights.sliding(2).map { x => @@ -366,7 +368,8 @@ abstract class RDD[T: ClassTag]( }.toArray } - def takeSample(withReplacement: Boolean, num: Int, seed: Int): Array[T] = { + def takeSample(withReplacement: Boolean, num: Int, seed: Long = Utils.random.nextLong): Array[T] = + { var fraction = 0.0 var total = 0 val multiplier = 3.0 diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index d333e2a88c18c..084a71c4caebd 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -46,6 +46,8 @@ import org.apache.spark.serializer.{DeserializationStream, SerializationStream, private[spark] object Utils extends Logging { val osName = System.getProperty("os.name") + + val random = new Random() /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { 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 2676558bfc86d..8da9a0da700e0 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -463,7 +463,13 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("takeSample") { val data = sc.parallelize(1 to 100, 2) - + + for (num <- List(5, 20, 100)) { + val sample = data.takeSample(withReplacement=false, num=num) + assert(sample.size === num) // Got exactly num elements + assert(sample.toSet.size === num) // Elements are distinct + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=false, 20, seed) assert(sample.size === 20) // Got exactly 20 elements @@ -481,6 +487,19 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(sample.size === 20) // Got exactly 20 elements assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") } + { + val sample = data.takeSample(withReplacement=true, num=20) + assert(sample.size === 20) // Got exactly 100 elements + assert(sample.toSet.size <= 20, "sampling with replacement returned all distinct elements") + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } + { + val sample = data.takeSample(withReplacement=true, num=100) + assert(sample.size === 100) // Got exactly 100 elements + // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=true, 100, seed) assert(sample.size === 100) // Got exactly 100 elements diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 91fc7e637e2c6..d73ab7006e9c7 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -30,6 +30,7 @@ from threading import Thread import warnings import heapq +from random import Random from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, pack_long @@ -332,7 +333,7 @@ def distinct(self): .reduceByKey(lambda x, _: x) \ .map(lambda (x, _): x) - def sample(self, withReplacement, fraction, seed): + def sample(self, withReplacement, fraction, seed=None): """ Return a sampled subset of this RDD (relies on numpy and falls back on default random generator if numpy is unavailable). @@ -344,7 +345,7 @@ def sample(self, withReplacement, fraction, seed): return self.mapPartitionsWithIndex(RDDSampler(withReplacement, fraction, seed).func, True) # this is ported from scala/spark/RDD.scala - def takeSample(self, withReplacement, num, seed): + def takeSample(self, withReplacement, num, seed=None): """ Return a fixed-size sampled subset of this RDD (currently requires numpy). @@ -381,13 +382,11 @@ def takeSample(self, withReplacement, num, seed): # 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 their initial size. # See: scala/spark/RDD.scala + rand = Random(seed) while len(samples) < total: - if seed > sys.maxint - 2: - seed = -1 - seed += 1 - samples = self.sample(withReplacement, fraction, seed).collect() + samples = self.sample(withReplacement, fraction, rand.randint(0, sys.maxint)).collect() - sampler = RDDSampler(withReplacement, fraction, seed+1) + sampler = RDDSampler(withReplacement, fraction, rand.randint(0, sys.maxint)) sampler.shuffle(samples) return samples[0:total] diff --git a/python/pyspark/rddsampler.py b/python/pyspark/rddsampler.py index aca2ef3b51e98..845a267e311c5 100644 --- a/python/pyspark/rddsampler.py +++ b/python/pyspark/rddsampler.py @@ -19,7 +19,7 @@ import random class RDDSampler(object): - def __init__(self, withReplacement, fraction, seed): + def __init__(self, withReplacement, fraction, seed=None): try: import numpy self._use_numpy = True @@ -27,7 +27,7 @@ def __init__(self, withReplacement, fraction, seed): print >> sys.stderr, "NumPy does not appear to be installed. Falling back to default random generator for sampling." self._use_numpy = False - self._seed = seed + self._seed = seed if seed is not None else random.randint(0, sys.maxint) self._withReplacement = withReplacement self._fraction = fraction self._random = None @@ -38,17 +38,14 @@ def initRandomGenerator(self, split): if self._use_numpy: import numpy self._random = numpy.random.RandomState(self._seed) - for _ in range(0, split): - # discard the next few values in the sequence to have a - # different seed for the different splits - self._random.randint(sys.maxint) else: - import random - random.seed(self._seed) - for _ in range(0, split): - # discard the next few values in the sequence to have a - # different seed for the different splits - random.randint(0, sys.maxint) + self._random = random.Random(self._seed) + + for _ in range(0, split): + # discard the next few values in the sequence to have a + # different seed for the different splits + self._random.randint(0, sys.maxint) + self._split = split self._rand_initialized = True @@ -59,7 +56,7 @@ def getUniformSample(self, split): if self._use_numpy: return self._random.random_sample() else: - return random.uniform(0.0, 1.0) + return self._random.uniform(0.0, 1.0) def getPoissonSample(self, split, mean): if not self._rand_initialized or split != self._split: @@ -73,26 +70,26 @@ def getPoissonSample(self, split, mean): num_arrivals = 1 cur_time = 0.0 - cur_time += random.expovariate(mean) + cur_time += self._random.expovariate(mean) if cur_time > 1.0: return 0 while(cur_time <= 1.0): - cur_time += random.expovariate(mean) + cur_time += self._random.expovariate(mean) num_arrivals += 1 return (num_arrivals - 1) def shuffle(self, vals): - if self._random == None or split != self._split: + if self._random == None: self.initRandomGenerator(0) # this should only ever called on the master so # the split does not matter if self._use_numpy: self._random.shuffle(vals) else: - random.shuffle(vals, self._random) + self._random.shuffle(vals, self._random.random) def func(self, split, iterator): if self._withReplacement: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 397473e178867..732708e146b04 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -168,7 +168,7 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode { def references = Set.empty } -case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: LogicalPlan) +case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: LogicalPlan) extends UnaryNode { def output = child.output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 6cb0e0f61f2d2..ca6e0a696405a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -256,10 +256,11 @@ class SchemaRDD( * @group Query */ @Experimental + override def sample( - fraction: Double, withReplacement: Boolean = true, - seed: Int = (math.random * 1000).toInt) = + fraction: Double, + seed: Long) = new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan)) /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index e4cf2020a1a93..d807187a5ffb8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -57,9 +57,9 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { * :: DeveloperApi :: */ @DeveloperApi -case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: SparkPlan) - extends UnaryNode { - +case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: SparkPlan) + extends UnaryNode +{ override def output = child.output // TODO: How to pick seed? From 526a518bf32ad55b926a26f16086f445fd0ae29f Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 24 Apr 2014 18:18:22 -0700 Subject: [PATCH 163/641] [SPARK-1592][streaming] Automatically remove streaming input blocks The raw input data is stored as blocks in BlockManagers. Earlier they were cleared by cleaner ttl. Now since streaming does not require cleaner TTL to be set, the block would not get cleared. This increases up the Spark's memory usage, which is not even accounted and shown in the Spark storage UI. It may cause the data blocks to spill over to disk, which eventually slows down the receiving of data (persisting to memory become bottlenecked by writing to disk). The solution in this PR is to automatically remove those blocks. The mechanism to keep track of which BlockRDDs (which has presents the raw data blocks as a RDD) can be safely cleared already exists. Just use it to explicitly remove blocks from BlockRDDs. Author: Tathagata Das Closes #512 from tdas/block-rdd-unpersist and squashes the following commits: d25e610 [Tathagata Das] Merge remote-tracking branch 'apache/master' into block-rdd-unpersist 5f46d69 [Tathagata Das] Merge remote-tracking branch 'apache/master' into block-rdd-unpersist 2c320cd [Tathagata Das] Updated configuration with spark.streaming.unpersist setting. 2d4b2fd [Tathagata Das] Automatically removed input blocks --- .../scala/org/apache/spark/rdd/BlockRDD.scala | 45 +++++++++-- docs/configuration.md | 7 +- .../org/apache/spark/streaming/Time.scala | 2 +- .../spark/streaming/dstream/DStream.scala | 16 +++- .../streaming/BasicOperationsSuite.scala | 76 ++++++++++++++++++- .../spark/streaming/InputStreamsSuite.scala | 13 ---- .../streaming/NetworkReceiverSuite.scala | 1 + 7 files changed, 135 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index e6c4a6d3794a0..c64da8804d166 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -19,24 +19,30 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{Partition, SparkContext, SparkEnv, TaskContext} +import org.apache.spark._ import org.apache.spark.storage.{BlockId, BlockManager} +import scala.Some private[spark] class BlockRDDPartition(val blockId: BlockId, idx: Int) extends Partition { val index = idx } private[spark] -class BlockRDD[T: ClassTag](sc: SparkContext, @transient blockIds: Array[BlockId]) +class BlockRDD[T: ClassTag](@transient sc: SparkContext, @transient val blockIds: Array[BlockId]) extends RDD[T](sc, Nil) { @transient lazy val locations_ = BlockManager.blockIdsToHosts(blockIds, SparkEnv.get) + @volatile private var _isValid = true - override def getPartitions: Array[Partition] = (0 until blockIds.size).map(i => { - new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition] - }).toArray + override def getPartitions: Array[Partition] = { + assertValid() + (0 until blockIds.size).map(i => { + new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition] + }).toArray + } override def compute(split: Partition, context: TaskContext): Iterator[T] = { + assertValid() val blockManager = SparkEnv.get.blockManager val blockId = split.asInstanceOf[BlockRDDPartition].blockId blockManager.get(blockId) match { @@ -47,7 +53,36 @@ class BlockRDD[T: ClassTag](sc: SparkContext, @transient blockIds: Array[BlockId } override def getPreferredLocations(split: Partition): Seq[String] = { + assertValid() locations_(split.asInstanceOf[BlockRDDPartition].blockId) } + + /** + * Remove the data blocks that this BlockRDD is made from. NOTE: This is an + * irreversible operation, as the data in the blocks cannot be recovered back + * once removed. Use it with caution. + */ + private[spark] def removeBlocks() { + blockIds.foreach { blockId => + sc.env.blockManager.master.removeBlock(blockId) + } + _isValid = false + } + + /** + * Whether this BlockRDD is actually usable. This will be false if the data blocks have been + * removed using `this.removeBlocks`. + */ + private[spark] def isValid: Boolean = { + _isValid + } + + /** Check if this BlockRDD is valid. If not valid, exception is thrown. */ + private[spark] def assertValid() { + if (!_isValid) { + throw new SparkException( + "Attempted to use %s after its blocks have been removed!".format(toString)) + } + } } diff --git a/docs/configuration.md b/docs/configuration.md index e7e1dd56cf124..8d3442625b475 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -469,10 +469,13 @@ Apart from these, the following properties are also available, and may be useful spark.streaming.unpersist - false + true Force RDDs generated and persisted by Spark Streaming to be automatically unpersisted from - Spark's memory. Setting this to true is likely to reduce Spark's RDD memory usage. + Spark's memory. The raw input data received by Spark Streaming is also automatically cleared. + Setting this to false will allow the raw data and persisted RDDs to be accessible outside the + streaming application as they will not be cleared automatically. But it comes at the cost of + higher memory usage in Spark. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala index 6a6b00a778b48..37b3b28fa01cb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala @@ -68,5 +68,5 @@ case class Time(private val millis: Long) { } object Time { - val ordering = Ordering.by((time: Time) => time.millis) + implicit val ordering = Ordering.by((time: Time) => time.millis) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index d393cc03cb33e..f69f69e0c44af 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -25,7 +25,7 @@ import scala.reflect.ClassTag import java.io.{IOException, ObjectInputStream, ObjectOutputStream} import org.apache.spark.Logging -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.MetadataCleaner import org.apache.spark.streaming._ @@ -340,13 +340,23 @@ abstract class DStream[T: ClassTag] ( * this to clear their own metadata along with the generated RDDs. */ private[streaming] def clearMetadata(time: Time) { + val unpersistData = ssc.conf.getBoolean("spark.streaming.unpersist", true) val oldRDDs = generatedRDDs.filter(_._1 <= (time - rememberDuration)) logDebug("Clearing references to old RDDs: [" + oldRDDs.map(x => s"${x._1} -> ${x._2.id}").mkString(", ") + "]") generatedRDDs --= oldRDDs.keys - if (ssc.conf.getBoolean("spark.streaming.unpersist", false)) { + if (unpersistData) { logDebug("Unpersisting old RDDs: " + oldRDDs.values.map(_.id).mkString(", ")) - oldRDDs.values.foreach(_.unpersist(false)) + oldRDDs.values.foreach { rdd => + rdd.unpersist(false) + // Explicitly remove blocks of BlockRDD + rdd match { + case b: BlockRDD[_] => + logInfo("Removing blocks of RDD " + b + " of time " + time) + b.removeBlocks() + case _ => + } + } } logDebug("Cleared " + oldRDDs.size + " RDDs that were older than " + (time - rememberDuration) + ": " + oldRDDs.keys.mkString(", ")) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 8aec27e39478a..4792ca1f8ae3e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.streaming import org.apache.spark.streaming.StreamingContext._ -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.SparkContext._ import util.ManualClock @@ -27,6 +27,8 @@ import org.apache.spark.{SparkContext, SparkConf} import org.apache.spark.streaming.dstream.{WindowedDStream, DStream} import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} import scala.reflect.ClassTag +import org.apache.spark.storage.StorageLevel +import scala.collection.mutable class BasicOperationsSuite extends TestSuiteBase { test("map") { @@ -450,6 +452,78 @@ class BasicOperationsSuite extends TestSuiteBase { assert(!stateStream.generatedRDDs.contains(Time(4000))) } + test("rdd cleanup - input blocks and persisted RDDs") { + // Actually receive data over through receiver to create BlockRDDs + + // Start the server + val testServer = new TestServer() + testServer.start() + + // Set up the streaming context and input streams + val ssc = new StreamingContext(conf, batchDuration) + val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) + val mappedStream = networkStream.map(_ + ".").persist() + val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] + val outputStream = new TestOutputStream(mappedStream, outputBuffer) + + outputStream.register() + ssc.start() + + // Feed data to the server to send to the network receiver + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val input = Seq(1, 2, 3, 4, 5, 6) + + val blockRdds = new mutable.HashMap[Time, BlockRDD[_]] + val persistentRddIds = new mutable.HashMap[Time, Int] + + def collectRddInfo() { // get all RDD info required for verification + networkStream.generatedRDDs.foreach { case (time, rdd) => + blockRdds(time) = rdd.asInstanceOf[BlockRDD[_]] + } + mappedStream.generatedRDDs.foreach { case (time, rdd) => + persistentRddIds(time) = rdd.id + } + } + + Thread.sleep(200) + for (i <- 0 until input.size) { + testServer.send(input(i).toString + "\n") + Thread.sleep(200) + clock.addToTime(batchDuration.milliseconds) + collectRddInfo() + } + + Thread.sleep(200) + collectRddInfo() + logInfo("Stopping server") + testServer.stop() + logInfo("Stopping context") + + // verify data has been received + assert(outputBuffer.size > 0) + assert(blockRdds.size > 0) + assert(persistentRddIds.size > 0) + + import Time._ + + val latestPersistedRddId = persistentRddIds(persistentRddIds.keySet.max) + val earliestPersistedRddId = persistentRddIds(persistentRddIds.keySet.min) + val latestBlockRdd = blockRdds(blockRdds.keySet.max) + val earliestBlockRdd = blockRdds(blockRdds.keySet.min) + // verify that the latest mapped RDD is persisted but the earliest one has been unpersisted + assert(ssc.sparkContext.persistentRdds.contains(latestPersistedRddId)) + assert(!ssc.sparkContext.persistentRdds.contains(earliestPersistedRddId)) + + // verify that the latest input blocks are present but the earliest blocks have been removed + assert(latestBlockRdd.isValid) + assert(latestBlockRdd.collect != null) + assert(!earliestBlockRdd.isValid) + earliestBlockRdd.blockIds.foreach { blockId => + assert(!ssc.sparkContext.env.blockManager.master.contains(blockId)) + } + ssc.stop() + } + /** Test cleanup of RDDs in DStream metadata */ def runCleanupTest[T: ClassTag]( conf2: SparkConf, diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 3bad871b5c580..b55b7834c90c1 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -42,8 +42,6 @@ import org.apache.spark.streaming.receiver.{ActorHelper, Receiver} class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { - val testPort = 9999 - test("socket input stream") { // Start the server val testServer = new TestServer() @@ -288,17 +286,6 @@ class TestServer(portToBind: Int = 0) extends Logging { def port = serverSocket.getLocalPort } -object TestServer { - def main(args: Array[String]) { - val s = new TestServer() - s.start() - while(true) { - Thread.sleep(1000) - s.send("hello") - } - } -} - /** This is an actor for testing actor input stream */ class TestActor(port: Int) extends Actor with ActorHelper { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala index 45304c76b0928..ff3619a59042d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala @@ -29,6 +29,7 @@ import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ +import scala.language.postfixOps /** Testsuite for testing the network receiver behavior */ class NetworkReceiverSuite extends FunSuite with Timeouts { From 4660991e679eda158a3ae8039b686eae197a71d1 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 24 Apr 2014 18:21:00 -0700 Subject: [PATCH 164/641] [SQL] Add support for parsing indexing into arrays in SQL. Author: Michael Armbrust Closes #518 from marmbrus/parseArrayIndex and squashes the following commits: afd2d6b [Michael Armbrust] 100 chars c3d6026 [Michael Armbrust] Add support for parsing indexing into arrays in SQL. --- .../apache/spark/sql/catalyst/SqlParser.scala | 10 +++++++--- .../org/apache/spark/sql/SQLQuerySuite.scala | 16 ++++++++++++++++ .../scala/org/apache/spark/sql/TestData.scala | 7 +++++++ 3 files changed, 30 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 13a19d0adf5e6..8c76a3aa96546 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst import scala.language.implicitConversions import scala.util.parsing.combinator.lexical.StdLexical import scala.util.parsing.combinator.syntactical.StandardTokenParsers +import scala.util.parsing.combinator.PackratParsers import scala.util.parsing.input.CharArrayReader.EofCh import org.apache.spark.sql.catalyst.analysis._ @@ -39,7 +40,7 @@ import org.apache.spark.sql.catalyst.types._ * This is currently included mostly for illustrative purposes. Users wanting more complete support * for a SQL like language should checkout the HiveQL support in the sql/hive sub-project. */ -class SqlParser extends StandardTokenParsers { +class SqlParser extends StandardTokenParsers with PackratParsers { def apply(input: String): LogicalPlan = { phrase(query)(new lexical.Scanner(input)) match { case Success(r, x) => r @@ -152,7 +153,7 @@ class SqlParser extends StandardTokenParsers { lexical.delimiters += ( "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")", - ",", ";", "%", "{", "}", ":" + ",", ";", "%", "{", "}", ":", "[", "]" ) protected def assignAliases(exprs: Seq[Expression]): Seq[NamedExpression] = { @@ -339,7 +340,10 @@ class SqlParser extends StandardTokenParsers { protected lazy val floatLit: Parser[String] = elem("decimal", _.isInstanceOf[lexical.FloatLit]) ^^ (_.chars) - protected lazy val baseExpression: Parser[Expression] = + protected lazy val baseExpression: PackratParser[Expression] = + expression ~ "[" ~ expression <~ "]" ^^ { + case base ~ _ ~ ordinal => GetItem(base, ordinal) + } | TRUE ^^^ Literal(true, BooleanType) | FALSE ^^^ Literal(false, BooleanType) | cast | diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 4c4fd6dbbedb4..dde957d715a28 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -28,6 +28,22 @@ class SQLQuerySuite extends QueryTest { // Make sure the tables are loaded. TestData + test("index into array") { + checkAnswer( + sql("SELECT data, data[0], data[0] + data[1], data[0 + 1] FROM arrayData"), + arrayData.map(d => (d.data, d.data(0), d.data(0) + d.data(1), d.data(1))).collect().toSeq) + } + + test("index into array of arrays") { + checkAnswer( + sql( + "SELECT nestedData, nestedData[0][0], nestedData[0][0] + nestedData[0][1] FROM arrayData"), + arrayData.map(d => + (d.nestedData, + d.nestedData(0)(0), + d.nestedData(0)(0) + d.nestedData(0)(1))).collect().toSeq) + } + test("agg") { checkAnswer( sql("SELECT a, SUM(b) FROM testData2 GROUP BY a"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 271b1d9fcacf8..002b7f0adafab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -66,4 +66,11 @@ object TestData { LowerCaseData(3, "c") :: LowerCaseData(4, "d") :: Nil) lowerCaseData.registerAsTable("lowerCaseData") + + case class ArrayData(data: Seq[Int], nestedData: Seq[Seq[Int]]) + val arrayData = + TestSQLContext.sparkContext.parallelize( + ArrayData(Seq(1,2,3), Seq(Seq(1,2,3))) :: + ArrayData(Seq(2,3,4), Seq(Seq(2,3,4))) :: Nil) + arrayData.registerAsTable("arrayData") } From 44da5ab2dea6dcf1e13d624784741141883870bb Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 24 Apr 2014 18:38:10 -0700 Subject: [PATCH 165/641] Spark 1489 Fix the HistoryServer view acls This allows the view acls set by the user to be enforced by the history server. It also fixes filters being applied properly. Author: Thomas Graves Closes #509 from tgravescs/SPARK-1489 and squashes the following commits: 869c186 [Thomas Graves] change to either acls enabled or disabled 0d8333c [Thomas Graves] Add history ui policy to allow acls to either use application set, history server force acls on, or off 65148b5 [Thomas Graves] SPARK-1489 Fix the HistoryServer view acls --- .../org/apache/spark/SecurityManager.scala | 26 ++++++-- .../spark/deploy/history/HistoryServer.scala | 14 ++++- .../scheduler/ApplicationEventListener.scala | 11 ++++ .../org/apache/spark/ui/JettyUtils.scala | 4 +- .../scala/org/apache/spark/ui/SparkUI.scala | 8 +++ .../apache/spark/SecurityManagerSuite.scala | 63 +++++++++++++++++++ docs/monitoring.md | 13 ++++ 7 files changed, 128 insertions(+), 11 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index b52f2d4f416b2..b4b0067801259 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -19,8 +19,6 @@ package org.apache.spark import java.net.{Authenticator, PasswordAuthentication} -import scala.collection.mutable.ArrayBuffer - import org.apache.hadoop.io.Text import org.apache.spark.deploy.SparkHadoopUtil @@ -139,13 +137,13 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { private val sparkSecretLookupKey = "sparkCookie" private val authOn = sparkConf.getBoolean("spark.authenticate", false) - private val uiAclsOn = sparkConf.getBoolean("spark.ui.acls.enable", false) + private var uiAclsOn = sparkConf.getBoolean("spark.ui.acls.enable", false) + private var viewAcls: Set[String] = _ // always add the current user and SPARK_USER to the viewAcls - private val aclUsers = ArrayBuffer[String](System.getProperty("user.name", ""), + private val defaultAclUsers = Seq[String](System.getProperty("user.name", ""), Option(System.getenv("SPARK_USER")).getOrElse("")) - aclUsers ++= sparkConf.get("spark.ui.view.acls", "").split(',') - private val viewAcls = aclUsers.map(_.trim()).filter(!_.isEmpty).toSet + setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) private val secretKey = generateSecretKey() logInfo("SecurityManager, is authentication enabled: " + authOn + @@ -170,6 +168,20 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { ) } + private[spark] def setViewAcls(defaultUsers: Seq[String], allowedUsers: String) { + viewAcls = (defaultUsers ++ allowedUsers.split(',')).map(_.trim()).filter(!_.isEmpty).toSet + logInfo("Changing view acls to: " + viewAcls.mkString(",")) + } + + private[spark] def setViewAcls(defaultUser: String, allowedUsers: String) { + setViewAcls(Seq[String](defaultUser), allowedUsers) + } + + private[spark] def setUIAcls(aclSetting: Boolean) { + uiAclsOn = aclSetting + logInfo("Changing acls enabled to: " + uiAclsOn) + } + /** * Generates or looks up the secret key. * @@ -222,6 +234,8 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { * @return true is the user has permission, otherwise false */ def checkUIViewPermissions(user: String): Boolean = { + logDebug("user=" + user + " uiAclsEnabled=" + uiAclsEnabled() + " viewAcls=" + + viewAcls.mkString(",")) if (uiAclsEnabled() && (user != null) && (!viewAcls.contains(user))) false else true } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index d7a3246bcfb47..1238bbf9da2fd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -168,17 +168,21 @@ class HistoryServer( * directory. If this file exists, the associated application is regarded to be completed, in * which case the server proceeds to render the SparkUI. Otherwise, the server does nothing. */ - private def renderSparkUI(logDir: FileStatus, logInfo: EventLoggingInfo) { + private def renderSparkUI(logDir: FileStatus, elogInfo: EventLoggingInfo) { val path = logDir.getPath val appId = path.getName - val replayBus = new ReplayListenerBus(logInfo.logPaths, fileSystem, logInfo.compressionCodec) + val replayBus = new ReplayListenerBus(elogInfo.logPaths, fileSystem, elogInfo.compressionCodec) val appListener = new ApplicationEventListener replayBus.addListener(appListener) - val ui = new SparkUI(conf, replayBus, appId, "/history/" + appId) + val appConf = conf.clone() + val appSecManager = new SecurityManager(appConf) + val ui = new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId) // Do not call ui.bind() to avoid creating a new server for each application replayBus.replay() if (appListener.applicationStarted) { + appSecManager.setUIAcls(HISTORY_UI_ACLS_ENABLED) + appSecManager.setViewAcls(appListener.sparkUser, appListener.viewAcls) attachSparkUI(ui) val appName = appListener.appName val sparkUser = appListener.sparkUser @@ -202,6 +206,7 @@ class HistoryServer( private def attachSparkUI(ui: SparkUI) { assert(serverInfo.isDefined, "HistoryServer must be bound before attaching SparkUIs") ui.getHandlers.foreach(attachHandler) + addFilters(ui.getHandlers, conf) } /** Detach a reconstructed UI from this server. Only valid after bind(). */ @@ -255,6 +260,9 @@ object HistoryServer { // The port to which the web UI is bound val WEB_UI_PORT = conf.getInt("spark.history.ui.port", 18080) + // set whether to enable or disable view acls for all applications + val HISTORY_UI_ACLS_ENABLED = conf.getBoolean("spark.history.ui.acls.enable", false) + val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR def main(argStrings: Array[String]) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index c1001227151a5..cd5d44ad4a7e6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -28,6 +28,8 @@ private[spark] class ApplicationEventListener extends SparkListener { var sparkUser = "" var startTime = -1L var endTime = -1L + var viewAcls = "" + var enableViewAcls = false def applicationStarted = startTime != -1 @@ -47,4 +49,13 @@ private[spark] class ApplicationEventListener extends SparkListener { override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { endTime = applicationEnd.time } + + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { + synchronized { + val environmentDetails = environmentUpdate.environmentDetails + val allProperties = environmentDetails("Spark Properties").toMap + viewAcls = allProperties.getOrElse("spark.ui.view.acls", "") + enableViewAcls = allProperties.getOrElse("spark.ui.acls.enable", "false").toBoolean + } + } } 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 fdeb15b5d058a..b3ac2320f3431 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -137,8 +137,8 @@ private[spark] object JettyUtils extends Logging { contextHandler } - /** Add security filters, if any, do the given list of ServletContextHandlers */ - private def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { + /** Add filters, if any, to the given list of ServletContextHandlers */ + def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) filters.foreach { case filter : String => diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 2fef1a635427c..097a1b81e1dd1 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -43,6 +43,14 @@ private[spark] class SparkUI( def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = this(null, conf, new SecurityManager(conf), listenerBus, appName, basePath) + def this( + conf: SparkConf, + securityManager: SecurityManager, + listenerBus: SparkListenerBus, + appName: String, + basePath: String) = + this(null, conf, securityManager, listenerBus, appName, basePath) + // If SparkContext is not provided, assume the associated application is not live val live = sc != null diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala new file mode 100644 index 0000000000000..e39093e24d68a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -0,0 +1,63 @@ +/* + * 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 + +import scala.collection.mutable.ArrayBuffer + +import org.scalatest.FunSuite + +class SecurityManagerSuite extends FunSuite { + + test("set security with conf") { + val conf = new SparkConf + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + conf.set("spark.ui.acls.enable", "true") + conf.set("spark.ui.view.acls", "user1,user2") + val securityManager = new SecurityManager(conf); + assert(securityManager.isAuthenticationEnabled() === true) + assert(securityManager.uiAclsEnabled() === true) + assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user2") === true) + assert(securityManager.checkUIViewPermissions("user3") === false) + } + + test("set security with api") { + val conf = new SparkConf + conf.set("spark.ui.view.acls", "user1,user2") + val securityManager = new SecurityManager(conf); + securityManager.setUIAcls(true) + assert(securityManager.uiAclsEnabled() === true) + securityManager.setUIAcls(false) + assert(securityManager.uiAclsEnabled() === false) + + // acls are off so doesn't matter what view acls set to + assert(securityManager.checkUIViewPermissions("user4") === true) + + securityManager.setUIAcls(true) + assert(securityManager.uiAclsEnabled() === true) + securityManager.setViewAcls(ArrayBuffer[String]("user5"), "user6,user7") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user5") === true) + assert(securityManager.checkUIViewPermissions("user6") === true) + assert(securityManager.checkUIViewPermissions("user7") === true) + assert(securityManager.checkUIViewPermissions("user8") === false) + assert(securityManager.checkUIViewPermissions(null) === true) + } +} + diff --git a/docs/monitoring.md b/docs/monitoring.md index 347a9b1f1a329..6f35fc37c4075 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -115,6 +115,19 @@ represents an application's event logs. This creates a web interface at Location of the kerberos keytab file for the History Server. + + spark.history.ui.acls.enable + false + + Specifies whether acls should be checked to authorize users viewing the applications. + If enabled, access control checks are made regardless of what the individual application had + set for spark.ui.acls.enable when the application was run. The application owner + will always have authorization to view their own application and any users specified via + spark.ui.view.acls when the application was run will also have authorization + to view that application. + If disabled, no access control checks are made. + + Note that in all of these UIs, the tables are sortable by clicking their headers, From 80429f3e2ab786d103297652922c3d8da3cf5a01 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 24 Apr 2014 18:56:57 -0700 Subject: [PATCH 166/641] [SPARK-1510] Spark Streaming metrics source for metrics system This pulls in changes made by @jerryshao in https://github.com/apache/spark/pull/424 and merges with the master. Author: jerryshao Author: Tathagata Das Closes #545 from tdas/streaming-metrics and squashes the following commits: 034b443 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-metrics fb3b0a5 [jerryshao] Modify according master update 21939f5 [jerryshao] Style changes according to style check error 976116b [jerryshao] Add StreamSource in StreamingContext for better monitoring through metrics system --- .../spark/streaming/StreamingContext.scala | 4 + .../spark/streaming/StreamingSource.scala | 73 +++++++++++++++++++ .../ui/StreamingJobProgressListener.scala | 3 +- 3 files changed, 79 insertions(+), 1 deletion(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 1c89543058211..e0677b795cb94 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -154,6 +154,10 @@ class StreamingContext private[streaming] ( private[streaming] val uiTab = new StreamingTab(this) + /** Register streaming source to metrics system */ + private val streamingSource = new StreamingSource(this) + SparkEnv.get.metricsSystem.registerSource(streamingSource) + /** Enumeration to identify current state of the StreamingContext */ private[streaming] object StreamingContextState extends Enumeration { type CheckpointState = Value diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala new file mode 100644 index 0000000000000..774adc3c23c21 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import org.apache.spark.metrics.source.Source +import org.apache.spark.streaming.ui.StreamingJobProgressListener + +private[streaming] class StreamingSource(ssc: StreamingContext) extends Source { + val metricRegistry = new MetricRegistry + val sourceName = "%s.StreamingMetrics".format(ssc.sparkContext.appName) + + val streamingListener = ssc.uiTab.listener + + private def registerGauge[T](name: String, f: StreamingJobProgressListener => T, + defaultValue: T) { + metricRegistry.register(MetricRegistry.name("streaming", name), new Gauge[T] { + override def getValue: T = Option(f(streamingListener)).getOrElse(defaultValue) + }) + } + + // Gauge for number of network receivers + registerGauge("receivers", _.numReceivers, 0) + + // Gauge for number of total completed batches + registerGauge("totalCompletedBatches", _.numTotalCompletedBatches, 0L) + + // Gauge for number of unprocessed batches + registerGauge("unprocessedBatches", _.numUnprocessedBatches, 0L) + + // Gauge for number of waiting batches + registerGauge("waitingBatches", _.waitingBatches.size, 0L) + + // Gauge for number of running batches + registerGauge("runningBatches", _.runningBatches.size, 0L) + + // Gauge for number of retained completed batches + registerGauge("retainedCompletedBatches", _.retainedCompletedBatches.size, 0L) + + // Gauge for last completed batch, useful for monitoring the streaming job's running status, + // displayed data -1 for any abnormal condition. + registerGauge("lastCompletedBatch_submissionTime", + _.lastCompletedBatch.map(_.submissionTime).getOrElse(-1L), -1L) + registerGauge("lastCompletedBatch_processStartTime", + _.lastCompletedBatch.flatMap(_.processingStartTime).getOrElse(-1L), -1L) + registerGauge("lastCompletedBatch_processEndTime", + _.lastCompletedBatch.flatMap(_.processingEndTime).getOrElse(-1L), -1L) + + // Gauge for last received batch, useful for monitoring the streaming job's running status, + // displayed data -1 for any abnormal condition. + registerGauge("lastReceivedBatch_submissionTime", + _.lastCompletedBatch.map(_.submissionTime).getOrElse(-1L), -1L) + registerGauge("lastReceivedBatch_processStartTime", + _.lastCompletedBatch.flatMap(_.processingStartTime).getOrElse(-1L), -1L) + registerGauge("lastReceivedBatch_processEndTime", + _.lastCompletedBatch.flatMap(_.processingEndTime).getOrElse(-1L), -1L) +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index bf637c1446314..14c33c728bfe1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -28,7 +28,8 @@ import org.apache.spark.streaming.scheduler.StreamingListenerBatchSubmitted import org.apache.spark.util.Distribution -private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends StreamingListener { +private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) + extends StreamingListener { private val waitingBatchInfos = new HashMap[Time, BatchInfo] private val runningBatchInfos = new HashMap[Time, BatchInfo] From ee6f7e22a449837864072e3cd2b6696005f134f1 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 24 Apr 2014 20:18:15 -0700 Subject: [PATCH 167/641] [SPARK-1615] Synchronize accesses to the LiveListenerBus' event queue Original poster is @zsxwing, who reported this bug in #516. Much of SparkListenerSuite relies on LiveListenerBus's `waitUntilEmpty()` method. As the name suggests, this waits until the event queue is empty. However, the following race condition could happen: (1) We dequeue an event (2) The queue is empty, we return true (even though the event has not been processed) (3) The test asserts something assuming that all listeners have finished executing (and fails) (4) The listeners receive and process the event This PR makes (1) and (4) atomic by synchronizing around it. To do that, however, we must avoid using `eventQueue.take`, which is blocking and will cause a deadlock if we synchronize around it. As a workaround, we use the non-blocking `eventQueue.poll` + a semaphore to provide the same semantics. This has been a possible race condition for a long time, but for some reason we've never run into it. Author: Andrew Or Closes #544 from andrewor14/stage-info-test-fix and squashes the following commits: 3cbe40c [Andrew Or] Merge github.com:apache/spark into stage-info-test-fix 56dbbcb [Andrew Or] Check if event is actually added before releasing semaphore eb486ae [Andrew Or] Synchronize accesses to the LiveListenerBus' event queue --- .../spark/scheduler/LiveListenerBus.scala | 38 ++++++++++++++----- .../spark/scheduler/SparkListenerSuite.scala | 5 ++- 2 files changed, 31 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index cbac4c13ca6fe..dec3316bf7745 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import java.util.concurrent.LinkedBlockingQueue +import java.util.concurrent.{LinkedBlockingQueue, Semaphore} import org.apache.spark.Logging @@ -36,16 +36,24 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) private var queueFullErrorMessageLogged = false private var started = false + + // A counter that represents the number of events produced and consumed in the queue + private val eventLock = new Semaphore(0) + private val listenerThread = new Thread("SparkListenerBus") { setDaemon(true) override def run() { while (true) { - val event = eventQueue.take - if (event == SparkListenerShutdown) { - // Get out of the while loop and shutdown the daemon thread - return + eventLock.acquire() + // Atomically remove and process this event + LiveListenerBus.this.synchronized { + val event = eventQueue.poll + if (event == SparkListenerShutdown) { + // Get out of the while loop and shutdown the daemon thread + return + } + Option(event).foreach(postToAll) } - postToAll(event) } } } @@ -67,7 +75,9 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { def post(event: SparkListenerEvent) { val eventAdded = eventQueue.offer(event) - if (!eventAdded && !queueFullErrorMessageLogged) { + if (eventAdded) { + eventLock.release() + } else if (!queueFullErrorMessageLogged) { logError("Dropping SparkListenerEvent because no remaining room in event queue. " + "This likely means one of the SparkListeners is too slow and cannot keep up with the " + "rate at which tasks are being started by the scheduler.") @@ -76,13 +86,13 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { } /** - * Waits until there are no more events in the queue, or until the specified time has elapsed. - * Used for testing only. Returns true if the queue has emptied and false is the specified time + * For testing only. Wait until there are no more events in the queue, or until the specified + * time has elapsed. Return true if the queue has emptied and false is the specified time * elapsed before the queue emptied. */ def waitUntilEmpty(timeoutMillis: Int): Boolean = { val finishTime = System.currentTimeMillis + timeoutMillis - while (!eventQueue.isEmpty) { + while (!queueIsEmpty) { if (System.currentTimeMillis > finishTime) { return false } @@ -93,6 +103,14 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { true } + /** + * Return whether the event queue is empty. + * + * The use of synchronized here guarantees that all events that once belonged to this queue + * have already been processed by all attached listeners, if this returns true. + */ + def queueIsEmpty: Boolean = synchronized { eventQueue.isEmpty } + def stop() { if (!started) { throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index ba048ced32a93..4e9fd07e68a21 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -29,7 +29,8 @@ import org.apache.spark.SparkContext._ import org.apache.spark.executor.TaskMetrics class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers - with BeforeAndAfter with BeforeAndAfterAll { + with BeforeAndAfter with BeforeAndAfterAll { + /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 @@ -37,7 +38,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc sc = new SparkContext("local", "SparkListenerSuite") } - override def afterAll { + override def afterAll() { System.clearProperty("spark.akka.frameSize") } From e53eb4f0159ebd4d72c4bbc3586fdfc66ccacab7 Mon Sep 17 00:00:00 2001 From: Ahir Reddy Date: Thu, 24 Apr 2014 20:21:10 -0700 Subject: [PATCH 168/641] [SPARK-986]: Job cancelation for PySpark * Additions to the PySpark API to cancel jobs * Monitor Thread in PythonRDD to kill Python workers if a task is interrupted Author: Ahir Reddy Closes #541 from ahirreddy/python-cancel and squashes the following commits: dfdf447 [Ahir Reddy] Changed success -> completed and made logging message clearer 6c860ab [Ahir Reddy] PR Comments 4b4100a [Ahir Reddy] Success flag adba6ed [Ahir Reddy] Destroy python workers 27a2f8f [Ahir Reddy] Start the writer thread... d422f7b [Ahir Reddy] Remove unnecesssary vals adda337 [Ahir Reddy] Busy wait on the ocntext.interrupted flag, and then kill the python worker d9e472f [Ahir Reddy] Revert "removed unnecessary vals" 5b9cae5 [Ahir Reddy] removed unnecessary vals 07b54d9 [Ahir Reddy] Fix canceling unit test 8ae9681 [Ahir Reddy] Don't interrupt worker 7722342 [Ahir Reddy] Monitor Thread for python workers db04e16 [Ahir Reddy] Added canceling api to PySpark --- .../scala/org/apache/spark/SparkEnv.scala | 8 +++ .../apache/spark/api/python/PythonRDD.scala | 30 ++++++++++- python/pyspark/context.py | 52 +++++++++++++++++-- 3 files changed, 86 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 915315ed74436..bea435ec34ce9 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -97,6 +97,14 @@ class SparkEnv ( pythonWorkers.getOrElseUpdate(key, new PythonWorkerFactory(pythonExec, envVars)).create() } } + + private[spark] + def destroyPythonWorker(pythonExec: String, envVars: Map[String, String]) { + synchronized { + val key = (pythonExec, envVars) + pythonWorkers(key).stop() + } + } } object SparkEnv extends Logging { 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 0d71fdbb03ec6..1498b017a793d 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 @@ -110,13 +110,41 @@ private[spark] class PythonRDD[T: ClassTag]( } }.start() + // Necessary to distinguish between a task that has failed and a task that is finished + @volatile var complete: Boolean = false + + // It is necessary to have a monitor thread for python workers if the user cancels with + // interrupts disabled. In that case we will need to explicitly kill the worker, otherwise the + // threads can block indefinitely. + new Thread(s"Worker Monitor for $pythonExec") { + override def run() { + // Kill the worker if it is interrupted or completed + // When a python task completes, the context is always set to interupted + while (!context.interrupted) { + Thread.sleep(2000) + } + if (!complete) { + try { + logWarning("Incomplete task interrupted: Attempting to kill Python Worker") + env.destroyPythonWorker(pythonExec, envVars.toMap) + } catch { + case e: Exception => + logError("Exception when trying to kill worker", e) + } + } + } + }.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) + context.addOnCompleteCallback{ () => + complete = true // Indicate that the task has completed successfully + context.interrupted = true + } // Return an iterator that read lines from the process's stdout val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize)) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index f63cc4a55fb98..c74dc5fd4f854 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -429,7 +429,7 @@ def _getJavaStorageLevel(self, storageLevel): storageLevel.deserialized, storageLevel.replication) - def setJobGroup(self, groupId, description): + def setJobGroup(self, groupId, description, interruptOnCancel=False): """ Assigns a group ID to all the jobs started by this thread until the group ID is set to a different value or cleared. @@ -437,8 +437,41 @@ def setJobGroup(self, groupId, description): Often, a unit of execution in an application consists of multiple Spark actions or jobs. Application programmers can use this method to group all those jobs together and give a group description. Once set, the Spark web UI will associate such jobs with this group. - """ - self._jsc.setJobGroup(groupId, description) + + The application can use L{SparkContext.cancelJobGroup} to cancel all + running jobs in this group. + + >>> import thread, threading + >>> from time import sleep + >>> result = "Not Set" + >>> lock = threading.Lock() + >>> def map_func(x): + ... sleep(100) + ... return x * x + >>> def start_job(x): + ... global result + ... try: + ... sc.setJobGroup("job_to_cancel", "some description") + ... result = sc.parallelize(range(x)).map(map_func).collect() + ... except Exception as e: + ... result = "Cancelled" + ... lock.release() + >>> def stop_job(): + ... sleep(5) + ... sc.cancelJobGroup("job_to_cancel") + >>> supress = lock.acquire() + >>> supress = thread.start_new_thread(start_job, (10,)) + >>> supress = thread.start_new_thread(stop_job, tuple()) + >>> supress = lock.acquire() + >>> print result + Cancelled + + If interruptOnCancel is set to true for the job group, then job cancellation will result + in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure + that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, + where HDFS may respond to Thread.interrupt() by marking nodes as dead. + """ + self._jsc.setJobGroup(groupId, description, interruptOnCancel) def setLocalProperty(self, key, value): """ @@ -460,6 +493,19 @@ def sparkUser(self): """ return self._jsc.sc().sparkUser() + def cancelJobGroup(self, groupId): + """ + Cancel active jobs for the specified group. See L{SparkContext.setJobGroup} + for more information. + """ + self._jsc.sc().cancelJobGroup(groupId) + + def cancelAllJobs(self): + """ + Cancel all jobs that have been scheduled or are running. + """ + self._jsc.sc().cancelAllJobs() + def _test(): import atexit import doctest From d5c6ae6cc3305b9aa3185486b5b6ba0a6e5aca90 Mon Sep 17 00:00:00 2001 From: tmalaska Date: Thu, 24 Apr 2014 20:31:17 -0700 Subject: [PATCH 169/641] SPARK-1584: Upgrade Flume dependency to 1.4.0 Updated the Flume dependency in the maven pom file and the scala build file. Author: tmalaska Closes #507 from tmalaska/master and squashes the following commits: 79492c8 [tmalaska] excluded all thrift 159c3f1 [tmalaska] fixed the flume pom file issues 5bf56a7 [tmalaska] Upgrade flume version --- external/flume/pom.xml | 6 +++++- project/SparkBuild.scala | 3 ++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/external/flume/pom.xml b/external/flume/pom.xml index f21963531574b..b84ca0c5e1dd0 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -61,12 +61,16 @@ org.apache.flume flume-ng-sdk - 1.2.0 + 1.4.0 org.jboss.netty netty + + org.apache.thrift + libthrift + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b8af2bbd2ef6a..9cbc188c4d485 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -313,6 +313,7 @@ object SparkBuild extends Build { val excludeCurator = ExclusionRule(organization = "org.apache.curator") val excludePowermock = ExclusionRule(organization = "org.powermock") val excludeFastutil = ExclusionRule(organization = "it.unimi.dsi") + val excludeThrift = ExclusionRule(organization = "org.apache.thrift") def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark", version: String = "0.9.0-incubating", crossVersion: String = "2.10"): Option[sbt.ModuleID] = { @@ -607,7 +608,7 @@ object SparkBuild extends Build { name := "spark-streaming-flume", previousArtifact := sparkPreviousArtifact("spark-streaming-flume"), libraryDependencies ++= Seq( - "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty) + "org.apache.flume" % "flume-ng-sdk" % "1.4.0" % "compile" excludeAll(excludeNetty, excludeThrift) ) ) From 968c0187a12f5ae4a696c02c1ff088e998ed7edd Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Thu, 24 Apr 2014 20:48:33 -0700 Subject: [PATCH 170/641] SPARK-1586 Windows build fixes Unfortunately, this is not exhaustive - particularly hive tests still fail due to path issues. Author: Mridul Muralidharan This patch had conflicts when merged, resolved by Committer: Matei Zaharia Closes #505 from mridulm/windows_fixes and squashes the following commits: ef12283 [Mridul Muralidharan] Move to org.apache.commons.lang3 for StringEscapeUtils. Earlier version was buggy appparently cdae406 [Mridul Muralidharan] Remove leaked changes from > 2G fix branch 3267f4b [Mridul Muralidharan] Fix build failures 35b277a [Mridul Muralidharan] Fix Scalastyle failures bc69d14 [Mridul Muralidharan] Change from hardcoded path separator 10c4d78 [Mridul Muralidharan] Use explicit encoding while using getBytes 1337abd [Mridul Muralidharan] fix classpath while running in windows --- bin/compute-classpath.cmd | 157 ++++++++++-------- .../org/apache/spark/SparkSaslClient.scala | 6 +- .../org/apache/spark/SparkSaslServer.scala | 8 +- .../apache/spark/api/python/PythonRDD.scala | 2 +- .../apache/spark/network/ReceiverTest.scala | 2 +- .../org/apache/spark/network/SenderTest.scala | 2 +- .../scala/org/apache/spark/rdd/PipedRDD.scala | 8 +- .../scala/org/apache/spark/util/Utils.scala | 24 ++- .../java/org/apache/spark/JavaAPISuite.java | 4 +- .../streaming/examples/MQTTWordCount.scala | 2 +- .../streaming/flume/FlumeStreamSuite.scala | 2 +- .../streaming/mqtt/MQTTInputDStream.scala | 2 +- .../org/apache/spark/repl/ReplSuite.scala | 6 +- .../spark/sql/columnar/ColumnType.scala | 6 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 44 ++++- .../spark/sql/hive/ScriptTransformation.scala | 2 +- .../org/apache/spark/sql/hive/TestHive.scala | 9 +- .../execution/BigDataBenchmarkSuite.scala | 2 +- .../hive/execution/HiveComparisonTest.scala | 5 +- .../execution/HiveCompatibilitySuite.scala | 6 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 2 +- 21 files changed, 185 insertions(+), 116 deletions(-) diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 4f60bff19cb93..065553eb31939 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -1,69 +1,88 @@ -@echo off - -rem -rem Licensed to the Apache Software Foundation (ASF) under one or more -rem contributor license agreements. See the NOTICE file distributed with -rem this work for additional information regarding copyright ownership. -rem The ASF licenses this file to You under the Apache License, Version 2.0 -rem (the "License"); you may not use this file except in compliance with -rem the License. You may obtain a copy of the License at -rem -rem http://www.apache.org/licenses/LICENSE-2.0 -rem -rem Unless required by applicable law or agreed to in writing, software -rem distributed under the License is distributed on an "AS IS" BASIS, -rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -rem See the License for the specific language governing permissions and -rem limitations under the License. -rem - -rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run" -rem script and the ExecutorRunner in standalone cluster mode. - -set SCALA_VERSION=2.10 - -rem Figure out where the Spark framework is installed -set FWDIR=%~dp0..\ - -rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" - -rem Build up classpath -set CLASSPATH=%FWDIR%conf -if exist "%FWDIR%RELEASE" ( - for %%d in ("%FWDIR%jars\spark-assembly*.jar") do ( - set ASSEMBLY_JAR=%%d - ) -) else ( - for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( - set ASSEMBLY_JAR=%%d - ) -) -set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% - -if "x%SPARK_TESTING%"=="x1" ( - rem Add test clases to path - set CLASSPATH=%CLASSPATH%;%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes - set CLASSPATH=%CLASSPATH%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes - set CLASSPATH=%CLASSPATH%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes - set CLASSPATH=%CLASSPATH%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes - set CLASSPATH=%CLASSPATH%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes -) - -rem Add hadoop conf dir - else FileSystem.*, etc fail -rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts -rem the configurtion files. -if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir - set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% -:no_hadoop_conf_dir - -if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir - set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% -:no_yarn_conf_dir - -rem A bit of a hack to allow calling this script within run2.cmd without seeing output -if "%DONT_PRINT_CLASSPATH%"=="1" goto exit - -echo %CLASSPATH% - -:exit +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run" +rem script and the ExecutorRunner in standalone cluster mode. + +set SCALA_VERSION=2.10 + +rem Figure out where the Spark framework is installed +set FWDIR=%~dp0..\ + +rem Load environment variables from conf\spark-env.cmd, if it exists +if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" + +rem Build up classpath +set CLASSPATH=%FWDIR%conf +if exist "%FWDIR%RELEASE" ( + for %%d in ("%FWDIR%jars\spark-assembly*.jar") do ( + set ASSEMBLY_JAR=%%d + ) +) else ( + for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( + set ASSEMBLY_JAR=%%d + ) +) + +set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% + +set SPARK_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%tools\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\classes + +set SPARK_TEST_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\test-classes + +if "x%SPARK_TESTING%"=="x1" ( + rem Add test clases to path - note, add SPARK_CLASSES and SPARK_TEST_CLASSES before CLASSPATH + rem so that local compilation takes precedence over assembled jar + set CLASSPATH=%SPARK_CLASSES%;%SPARK_TEST_CLASSES%;%CLASSPATH% +) + +rem Add hadoop conf dir - else FileSystem.*, etc fail +rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts +rem the configurtion files. +if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir + set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% +:no_hadoop_conf_dir + +if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir + set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% +:no_yarn_conf_dir + +rem A bit of a hack to allow calling this script within run2.cmd without seeing output +if "%DONT_PRINT_CLASSPATH%"=="1" goto exit + +echo %CLASSPATH% + +:exit diff --git a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala index 5b14c4291d91a..65003b6ac6a0a 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala @@ -111,10 +111,10 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg CallbackHandler { private val userName: String = - SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes()) + SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes("utf-8")) private val secretKey = securityMgr.getSecretKey() - private val userPassword: Array[Char] = - SparkSaslServer.encodePassword(if (secretKey != null) secretKey.getBytes() else "".getBytes()) + private val userPassword: Array[Char] = SparkSaslServer.encodePassword( + if (secretKey != null) secretKey.getBytes("utf-8") else "".getBytes("utf-8")) /** * Implementation used to respond to SASL request from the server. diff --git a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala index 6161a6fb7ae85..f6b0a9132aca4 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala @@ -89,7 +89,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi extends CallbackHandler { private val userName: String = - SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes()) + SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes("utf-8")) override def handle(callbacks: Array[Callback]) { logDebug("In the sasl server callback handler") @@ -101,7 +101,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi case pc: PasswordCallback => { logDebug("handle: SASL server callback: setting userPassword") val password: Array[Char] = - SparkSaslServer.encodePassword(securityMgr.getSecretKey().getBytes()) + SparkSaslServer.encodePassword(securityMgr.getSecretKey().getBytes("utf-8")) pc.setPassword(password) } case rc: RealmCallback => { @@ -159,7 +159,7 @@ private[spark] object SparkSaslServer { * @return Base64-encoded string */ def encodeIdentifier(identifier: Array[Byte]): String = { - new String(Base64.encodeBase64(identifier)) + new String(Base64.encodeBase64(identifier), "utf-8") } /** @@ -168,7 +168,7 @@ private[spark] object SparkSaslServer { * @return password as a char array. */ def encodePassword(password: Array[Byte]): Array[Char] = { - new String(Base64.encodeBase64(password)).toCharArray() + new String(Base64.encodeBase64(password), "utf-8").toCharArray() } } 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 1498b017a793d..672c344a56597 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 @@ -187,7 +187,7 @@ private[spark] class PythonRDD[T: ClassTag]( val exLength = stream.readInt() val obj = new Array[Byte](exLength) stream.readFully(obj) - throw new PythonException(new String(obj), readerException) + throw new PythonException(new String(obj, "utf-8"), readerException) case SpecialLengths.END_OF_DATA_SECTION => // We've finished the data section of the output, but we can still // read some accumulator updates: diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 9dc51e0d401f8..53a6038a9b59e 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -28,7 +28,7 @@ private[spark] object ReceiverTest { manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { /* println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis) */ - val buffer = ByteBuffer.wrap("response".getBytes) + val buffer = ByteBuffer.wrap("response".getBytes("utf-8")) Some(Message.createBufferMessage(buffer, msg.id)) }) Thread.currentThread.join() diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index 14c094c6177d5..b8ea7c2cff9a2 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -54,7 +54,7 @@ private[spark] object SenderTest { val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) .map { response => val buffer = response.asInstanceOf[BufferMessage].buffers(0) - new String(buffer.array) + new String(buffer.array, "utf-8") }.getOrElse("none") val finishTime = System.currentTimeMillis diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index e441d4a40ccd2..5d77d37378458 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -87,10 +87,10 @@ private[spark] class PipedRDD[T: ClassTag]( // When spark.worker.separated.working.directory option is turned on, each // task will be run in separate directory. This should be resolve file // access conflict issue - val taskDirectory = "./tasks/" + java.util.UUID.randomUUID.toString + val taskDirectory = "tasks" + File.separator + java.util.UUID.randomUUID.toString var workInTaskDirectory = false logDebug("taskDirectory = " + taskDirectory) - if (separateWorkingDir == true) { + if (separateWorkingDir) { val currentDir = new File(".") logDebug("currentDir = " + currentDir.getAbsolutePath()) val taskDirFile = new File(taskDirectory) @@ -106,13 +106,13 @@ private[spark] class PipedRDD[T: ClassTag]( for (file <- currentDir.list(tasksDirFilter)) { val fileWithDir = new File(currentDir, file) Utils.symlink(new File(fileWithDir.getAbsolutePath()), - new File(taskDirectory + "/" + fileWithDir.getName())) + new File(taskDirectory + File.separator + fileWithDir.getName())) } pb.directory(taskDirFile) workInTaskDirectory = true } catch { case e: Exception => logError("Unable to setup task working directory: " + e.getMessage + - " (" + taskDirectory + ")") + " (" + taskDirectory + ")", e) } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 084a71c4caebd..8351f7156a5e4 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -30,6 +30,7 @@ import scala.io.Source import scala.reflect.ClassTag import com.google.common.io.Files +import org.apache.commons.lang.SystemUtils import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.json4s._ @@ -45,10 +46,13 @@ import org.apache.spark.serializer.{DeserializationStream, SerializationStream, */ private[spark] object Utils extends Logging { - val osName = System.getProperty("os.name") - val random = new Random() + def sparkBin(sparkHome: String, which: String): File = { + val suffix = if (SystemUtils.IS_OS_WINDOWS) ".cmd" else "" + new File(sparkHome + File.separator + "bin", which + suffix) + } + /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -605,7 +609,7 @@ private[spark] object Utils extends Logging { */ def isSymlink(file: File): Boolean = { if (file == null) throw new NullPointerException("File must not be null") - if (osName.startsWith("Windows")) return false + if (SystemUtils.IS_OS_WINDOWS) return false val fileInCanonicalDir = if (file.getParent() == null) { file } else { @@ -1008,10 +1012,18 @@ private[spark] object Utils extends Logging { if (dst.isAbsolute()) { throw new IOException("Destination must be relative") } - val linkCmd = if (osName.startsWith("Windows")) "copy" else "ln -sf" + var cmdSuffix = "" + val linkCmd = if (SystemUtils.IS_OS_WINDOWS) { + // refer to http://technet.microsoft.com/en-us/library/cc771254.aspx + cmdSuffix = " /s /e /k /h /y /i" + "cmd /c xcopy " + } else { + cmdSuffix = "" + "ln -sf " + } import scala.sys.process._ - (linkCmd + " " + src.getAbsolutePath() + " " + dst.getPath()) lines_! ProcessLogger(line => - (logInfo(line))) + (linkCmd + src.getAbsolutePath() + " " + dst.getPath() + cmdSuffix) lines_! + ProcessLogger(line => (logInfo(line))) } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 8d2e9f1846343..76c6f5af82a5e 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -610,8 +610,8 @@ public void textFiles() throws IOException { @Test public void wholeTextFiles() throws IOException { - byte[] content1 = "spark is easy to use.\n".getBytes(); - byte[] content2 = "spark is also easy to use.\n".getBytes(); + byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); + byte[] content2 = "spark is also easy to use.\n".getBytes("utf-8"); File tempDir = Files.createTempDir(); String tempDirName = tempDir.getAbsolutePath(); diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index 3d3c27ed78204..62aef0fb47107 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -56,7 +56,7 @@ object MQTTPublisher { val msg: String = "hello mqtt demo for spark streaming" while (true) { - val message: MqttMessage = new MqttMessage(String.valueOf(msg).getBytes()) + val message: MqttMessage = new MqttMessage(String.valueOf(msg).getBytes("utf-8")) msgtopic.publish(message) println("Published data. topic: " + msgtopic.getName() + " Message: " + message) } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 78603200d2d8a..dd287d0ef90a0 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -57,7 +57,7 @@ class FlumeStreamSuite extends TestSuiteBase { for (i <- 0 until input.size) { val event = new AvroFlumeEvent - event.setBody(ByteBuffer.wrap(input(i).toString.getBytes())) + event.setBody(ByteBuffer.wrap(input(i).toString.getBytes("utf-8"))) event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header")) client.append(event) Thread.sleep(500) diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala index 0beee8b4157ae..77661f71ada21 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala @@ -92,7 +92,7 @@ class MQTTReceiver( // Handles Mqtt message override def messageArrived(arg0: String, arg1: MqttMessage) { - store(new String(arg1.getPayload())) + store(new String(arg1.getPayload(),"utf-8")) } override def deliveryComplete(arg0: IMqttDeliveryToken) { diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 4155007c6d337..e33f4f9803054 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -25,6 +25,7 @@ import scala.collection.mutable.ArrayBuffer import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.spark.SparkContext +import org.apache.commons.lang3.StringEscapeUtils class ReplSuite extends FunSuite { @@ -185,11 +186,12 @@ class ReplSuite extends FunSuite { out.close() val output = runInterpreter("local", """ - |var file = sc.textFile("%s/input").cache() + |var file = sc.textFile("%s").cache() |file.count() |file.count() |file.count() - """.stripMargin.format(tempDir.getAbsolutePath)) + """.stripMargin.format(StringEscapeUtils.escapeJava( + tempDir.getAbsolutePath + File.separator + "input"))) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Long = 3", output) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index 5be76890afe31..4cd52d8288137 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -200,10 +200,10 @@ private[sql] object SHORT extends NativeColumnType(ShortType, 6, 2) { } private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { - override def actualSize(v: String): Int = v.getBytes.length + 4 + override def actualSize(v: String): Int = v.getBytes("utf-8").length + 4 override def append(v: String, buffer: ByteBuffer) { - val stringBytes = v.getBytes() + val stringBytes = v.getBytes("utf-8") buffer.putInt(stringBytes.length).put(stringBytes, 0, stringBytes.length) } @@ -211,7 +211,7 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { val length = buffer.getInt() val stringBytes = new Array[Byte](length) buffer.get(stringBytes, 0, length) - new String(stringBytes) + new String(stringBytes, "utf-8") } override def setField(row: MutableRow, ordinal: Int, value: String) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 1d3608ed2d9ab..325173cf95fdf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -58,7 +58,7 @@ class ColumnTypeSuite extends FunSuite { checkActualSize(DOUBLE, Double.MaxValue, 8) checkActualSize(FLOAT, Float.MaxValue, 4) checkActualSize(BOOLEAN, true, 1) - checkActualSize(STRING, "hello", 4 + 5) + checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) val binary = Array.fill[Byte](4)(0: Byte) checkActualSize(BINARY, binary, 4 + 4) @@ -91,14 +91,16 @@ class ColumnTypeSuite extends FunSuite { testNativeColumnType[StringType.type]( STRING, (buffer: ByteBuffer, string: String) => { - val bytes = string.getBytes() - buffer.putInt(bytes.length).put(string.getBytes) + + val bytes = string.getBytes("utf-8") + buffer.putInt(bytes.length) + buffer.put(bytes) }, (buffer: ByteBuffer) => { val length = buffer.getInt() val bytes = new Array[Byte](length) - buffer.get(bytes, 0, length) - new String(bytes) + buffer.get(bytes) + new String(bytes, "utf-8") }) testColumnType[BinaryType.type, Array[Byte]]( @@ -161,9 +163,13 @@ class ColumnTypeSuite extends FunSuite { buffer.rewind() seq.foreach { expected => + println("buffer = " + buffer + ", expected = " + expected) + val extracted = columnType.extract(buffer) assert( - expected === columnType.extract(buffer), - "Extracted value didn't equal to the original one") + expected === extracted, + "Extracted value didn't equal to the original one. " + + hexDump(expected) + " != " + hexDump(extracted) + + ", buffer = " + dumpBuffer(buffer.duplicate().rewind().asInstanceOf[ByteBuffer])) } } @@ -179,4 +185,28 @@ class ColumnTypeSuite extends FunSuite { } } } + + private def hexDump(value: Any): String = { + if (value.isInstanceOf[String]) { + val sb = new StringBuilder() + for (ch <- value.asInstanceOf[String].toCharArray) { + sb.append(Integer.toHexString(ch & 0xffff)).append(' ') + } + if (! sb.isEmpty) sb.setLength(sb.length - 1) + sb.toString() + } else { + // for now .. + hexDump(value.toString) + } + } + + private def dumpBuffer(buff: ByteBuffer): Any = { + val sb = new StringBuilder() + while (buff.hasRemaining) { + val b = buff.get() + sb.append(Integer.toHexString(b & 0xff)).append(' ') + } + if (! sb.isEmpty) sb.setLength(sb.length - 1) + sb.toString() + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala index 610fa9cb843c0..8258ee5fef0eb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala @@ -71,7 +71,7 @@ case class ScriptTransformation( iter .map(outputProjection) // TODO: Use SerDe - .map(_.mkString("", "\t", "\n").getBytes).foreach(outputStream.write) + .map(_.mkString("", "\t", "\n").getBytes("utf-8")).foreach(outputStream.write) outputStream.close() readerThread.join() outputLines.toIterator diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 74110ee27b76c..3ad66a3d7f45f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -100,14 +100,15 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { hiveFilesTemp.delete() hiveFilesTemp.mkdir() - val inRepoTests = if (System.getProperty("user.dir").endsWith("sql/hive")) { - new File("src/test/resources/") + val inRepoTests = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { + new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) } else { - new File("sql/hive/src/test/resources") + new File("sql" + File.separator + "hive" + File.separator + "src" + File.separator + "test" + + File.separator + "resources") } def getHiveFile(path: String): File = { - val stripped = path.replaceAll("""\.\.\/""", "") + val stripped = path.replaceAll("""\.\.\/""", "").replace('/', File.separatorChar) hiveDevHome .map(new File(_, stripped)) .filter(_.exists) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala index 9b9a823b6e95b..42a82c1fbf5c7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.hive.test.TestHive._ * https://amplab.cs.berkeley.edu/benchmark/ */ class BigDataBenchmarkSuite extends HiveComparisonTest { - val testDataDirectory = new File("target/big-data-benchmark-testdata") + val testDataDirectory = new File("target" + File.separator + "big-data-benchmark-testdata") val testTables = Seq( TestTable( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index ea17e6e93b41c..edff38b901073 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -78,7 +78,8 @@ abstract class HiveComparisonTest .map(name => new File(targetDir, s"$suiteName.$name")) /** The local directory with cached golden answer will be stored. */ - protected val answerCache = new File("src/test/resources/golden") + protected val answerCache = new File("src" + File.separator + "test" + + File.separator + "resources" + File.separator + "golden") if (!answerCache.exists) { answerCache.mkdir() } @@ -120,7 +121,7 @@ abstract class HiveComparisonTest protected val cacheDigest = java.security.MessageDigest.getInstance("MD5") protected def getMd5(str: String): String = { val digest = java.security.MessageDigest.getInstance("MD5") - digest.update(str.getBytes) + digest.update(str.getBytes("utf-8")) new java.math.BigInteger(1, digest.digest).toString(16) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index dfe88b960b639..0bb76f31c373d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.execution +import java.io.File + import org.scalatest.BeforeAndAfter import org.apache.spark.sql.hive.test.TestHive @@ -26,7 +28,9 @@ import org.apache.spark.sql.hive.test.TestHive */ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // TODO: bundle in jar files... get from classpath - lazy val hiveQueryDir = TestHive.getHiveFile("ql/src/test/queries/clientpositive") + lazy val hiveQueryDir = TestHive.getHiveFile("ql" + File.separator + "src" + + File.separator + "test" + File.separator + "queries" + File.separator + "clientpositive") + def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) override def beforeAll() { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 832d45b3ad10e..718cb19f57261 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -64,7 +64,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { override def addSecretKeyToUserCredentials(key: String, secret: String) { val creds = new Credentials() - creds.addSecretKey(new Text(key), secret.getBytes()) + creds.addSecretKey(new Text(key), secret.getBytes("utf-8")) addCurrentUserCredentials(creds) } From cd12dd9bde91303d0341180e5f70d2a03d6b65b6 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 24 Apr 2014 21:34:37 -0700 Subject: [PATCH 171/641] [SPARK-1617] and [SPARK-1618] Improvements to streaming ui and bug fix to socket receiver 1617: These changes expose the receiver state (active or inactive) and last error in the UI 1618: If the socket receiver cannot connect in the first attempt, it should try to restart after a delay. That was broken, as the thread that restarts (hence, stops) the receiver waited on Thread.join on itself! Author: Tathagata Das Closes #540 from tdas/streaming-ui-fix and squashes the following commits: e469434 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-ui-fix dbddf75 [Tathagata Das] Style fix. 66df1a5 [Tathagata Das] Merge remote-tracking branch 'apache/master' into streaming-ui-fix ad98bc9 [Tathagata Das] Refactored streaming listener to use ReceiverInfo. d7f849c [Tathagata Das] Revert "Moved BatchInfo from streaming.scheduler to streaming.ui" 5c80919 [Tathagata Das] Moved BatchInfo from streaming.scheduler to streaming.ui da244f6 [Tathagata Das] Fixed socket receiver as well as made receiver state and error visible in the streamign UI. --- .../scala/org/apache/spark/ui/UIUtils.scala | 2 +- .../dstream/SocketInputDStream.scala | 49 +++++++--------- .../streaming/receiver/ActorReceiver.scala | 12 +++- .../spark/streaming/receiver/Receiver.scala | 5 +- .../streaming/receiver/ReceiverMessage.scala | 4 +- .../receiver/ReceiverSupervisor.scala | 58 +++++++++++-------- .../receiver/ReceiverSupervisorImpl.scala | 24 +++++--- .../spark/streaming/scheduler/BatchInfo.scala | 3 + .../streaming/scheduler/ReceiverInfo.scala | 37 ++++++++++++ .../streaming/scheduler/ReceiverTracker.scala | 40 ++++++++----- .../scheduler/StreamingListener.scala | 25 +++++++- .../ui/StreamingJobProgressListener.scala | 18 +++++- .../spark/streaming/ui/StreamingPage.scala | 20 +++++-- .../streaming/NetworkReceiverSuite.scala | 8 ++- .../streaming/StreamingListenerSuite.scala | 15 ++--- 15 files changed, 217 insertions(+), 103 deletions(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index cf987a1ab02c3..a3d6a1821245b 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -122,7 +122,7 @@ private[spark] object UIUtils extends Logging { } } if (unit.isEmpty) { - "%d".formatLocal(Locale.US, value) + "%d".formatLocal(Locale.US, value.toInt) } else { "%.1f%s".formatLocal(Locale.US, value, unit) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala index 1e32727eacfa3..8b72bcf20653d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala @@ -50,49 +50,42 @@ class SocketReceiver[T: ClassTag]( storageLevel: StorageLevel ) extends Receiver[T](storageLevel) with Logging { - var socket: Socket = null - var receivingThread: Thread = null - def onStart() { - receivingThread = new Thread("Socket Receiver") { - override def run() { - connect() - receive() - } - } - receivingThread.start() + // Start the thread that receives data over a connection + new Thread("Socket Receiver") { + setDaemon(true) + override def run() { receive() } + }.start() } def onStop() { - if (socket != null) { - socket.close() - } - socket = null - if (receivingThread != null) { - receivingThread.join() - } + // There is nothing much to do as the thread calling receive() + // is designed to stop by itself isStopped() returns false } - def connect() { + /** Create a socket connection and receive data until receiver is stopped */ + def receive() { + var socket: Socket = null try { logInfo("Connecting to " + host + ":" + port) socket = new Socket(host, port) - } catch { - case e: Exception => - restart("Could not connect to " + host + ":" + port, e) - } - } - - def receive() { - try { logInfo("Connected to " + host + ":" + port) val iterator = bytesToObjects(socket.getInputStream()) while(!isStopped && iterator.hasNext) { store(iterator.next) } + logInfo("Stopped receiving") + restart("Retrying connecting to " + host + ":" + port) } catch { - case e: Exception => - restart("Error receiving data from socket", e) + case e: java.net.ConnectException => + restart("Error connecting to " + host + ":" + port, e) + case t: Throwable => + restart("Error receiving data", t) + } finally { + if (socket != null) { + socket.close() + logInfo("Closed socket to " + host + ":" + port) + } } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala index 821cf19481d44..743be58950c09 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala @@ -28,8 +28,13 @@ import akka.actor.SupervisorStrategy.{Escalate, Restart} import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.storage.StorageLevel import java.nio.ByteBuffer +import org.apache.spark.annotation.DeveloperApi -/** A helper with set of defaults for supervisor strategy */ +/** + * :: DeveloperApi :: + * A helper with set of defaults for supervisor strategy + */ +@DeveloperApi object ActorSupervisorStrategy { val defaultStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange = @@ -40,6 +45,7 @@ object ActorSupervisorStrategy { } /** + * :: DeveloperApi :: * A receiver trait to be mixed in with your Actor to gain access to * the API for pushing received data into Spark Streaming for being processed. * @@ -61,6 +67,7 @@ object ActorSupervisorStrategy { * to ensure the type safety, i.e parametrized type of push block and InputDStream * should be same. */ +@DeveloperApi trait ActorHelper { self: Actor => // to ensure that this can be added to Actor classes only @@ -92,10 +99,12 @@ trait ActorHelper { } /** + * :: DeveloperApi :: * Statistics for querying the supervisor about state of workers. Used in * conjunction with `StreamingContext.actorStream` and * [[org.apache.spark.streaming.receiver.ActorHelper]]. */ +@DeveloperApi case class Statistics(numberOfMsgs: Int, numberOfWorkers: Int, numberOfHiccups: Int, @@ -188,4 +197,3 @@ private[streaming] class ActorReceiver[T: ClassTag]( supervisor ! PoisonPill } } - diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala index 44eecf1dd2567..524c1b8d8ce46 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -23,8 +23,10 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import org.apache.spark.storage.StorageLevel +import org.apache.spark.annotation.DeveloperApi /** + * :: DeveloperApi :: * Abstract class of a receiver that can be run on worker nodes to receive external data. A * custom receiver can be defined by defining the functions onStart() and onStop(). onStart() * should define the setup steps necessary to start receiving data, @@ -51,6 +53,7 @@ import org.apache.spark.storage.StorageLevel * } * }}} */ +@DeveloperApi abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable { /** @@ -198,7 +201,7 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable /** Check if receiver has been marked for stopping. */ def isStopped(): Boolean = { - !executor.isReceiverStarted() + executor.isReceiverStopped() } /** Get unique identifier of this receiver. */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala index 6ab3ca6ea5fa6..bf39d1e891cae 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala @@ -18,6 +18,6 @@ package org.apache.spark.streaming.receiver /** Messages sent to the NetworkReceiver. */ -private[streaming] sealed trait NetworkReceiverMessage -private[streaming] object StopReceiver extends NetworkReceiverMessage +private[streaming] sealed trait ReceiverMessage +private[streaming] object StopReceiver extends ReceiverMessage diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 256b3335e49aa..09be3a50d2dfa 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -88,15 +88,29 @@ private[streaming] abstract class ReceiverSupervisor( /** Report errors. */ def reportError(message: String, throwable: Throwable) - /** Start the executor */ + /** Called when supervisor is started */ + protected def onStart() { } + + /** Called when supervisor is stopped */ + protected def onStop(message: String, error: Option[Throwable]) { } + + /** Called when receiver is started */ + protected def onReceiverStart() { } + + /** Called when receiver is stopped */ + protected def onReceiverStop(message: String, error: Option[Throwable]) { } + + /** Start the supervisor */ def start() { + onStart() startReceiver() } - /** Mark the executor and the receiver for stopping */ + /** Mark the supervisor and the receiver for stopping */ def stop(message: String, error: Option[Throwable]) { stoppingError = error.orNull stopReceiver(message, error) + onStop(message, error) stopLatch.countDown() } @@ -104,6 +118,8 @@ private[streaming] abstract class ReceiverSupervisor( def startReceiver(): Unit = synchronized { try { logInfo("Starting receiver") + receiver.onStart() + logInfo("Called receiver onStart") onReceiverStart() receiverState = Started } catch { @@ -115,7 +131,10 @@ private[streaming] abstract class ReceiverSupervisor( /** Stop receiver */ def stopReceiver(message: String, error: Option[Throwable]): Unit = synchronized { try { + logInfo("Stopping receiver with message: " + message + ": " + error.getOrElse("")) receiverState = Stopped + receiver.onStop() + logInfo("Called receiver onStop") onReceiverStop(message, error) } catch { case t: Throwable => @@ -130,41 +149,32 @@ private[streaming] abstract class ReceiverSupervisor( /** Restart receiver with delay */ def restartReceiver(message: String, error: Option[Throwable], delay: Int) { - logWarning("Restarting receiver with delay " + delay + " ms: " + message, - error.getOrElse(null)) - stopReceiver("Restarting receiver with delay " + delay + "ms: " + message, error) - future { + Future { + logWarning("Restarting receiver with delay " + delay + " ms: " + message, + error.getOrElse(null)) + stopReceiver("Restarting receiver with delay " + delay + "ms: " + message, error) logDebug("Sleeping for " + delay) Thread.sleep(delay) - logDebug("Starting receiver again") + logInfo("Starting receiver again") startReceiver() logInfo("Receiver started again") } } - /** Called when the receiver needs to be started */ - protected def onReceiverStart(): Unit = synchronized { - // Call user-defined onStart() - logInfo("Calling receiver onStart") - receiver.onStart() - logInfo("Called receiver onStart") - } - - /** Called when the receiver needs to be stopped */ - protected def onReceiverStop(message: String, error: Option[Throwable]): Unit = synchronized { - // Call user-defined onStop() - logInfo("Calling receiver onStop") - receiver.onStop() - logInfo("Called receiver onStop") - } - /** Check if receiver has been marked for stopping */ def isReceiverStarted() = { logDebug("state = " + receiverState) receiverState == Started } - /** Wait the thread until the executor is stopped */ + /** Check if receiver has been marked for stopping */ + def isReceiverStopped() = { + logDebug("state = " + receiverState) + receiverState == Stopped + } + + + /** Wait the thread until the supervisor is stopped */ def awaitTermination() { stopLatch.await() logInfo("Waiting for executor stop is over") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 2a3521bd46ae7..ce8316bb14891 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -79,6 +79,8 @@ private[streaming] class ReceiverSupervisorImpl( logInfo("Received stop signal") stop("Stopped by driver", None) } + + def ref = self }), "Receiver-" + streamId + "-" + System.currentTimeMillis()) /** Unique block ids if one wants to add blocks directly */ @@ -154,14 +156,23 @@ private[streaming] class ReceiverSupervisorImpl( logWarning("Reported error " + message + " - " + error) } - override def onReceiverStart() { + override protected def onStart() { blockGenerator.start() - super.onReceiverStart() } - override def onReceiverStop(message: String, error: Option[Throwable]) { - super.onReceiverStop(message, error) + override protected def onStop(message: String, error: Option[Throwable]) { blockGenerator.stop() + env.actorSystem.stop(actor) + } + + override protected def onReceiverStart() { + val msg = RegisterReceiver( + streamId, receiver.getClass.getSimpleName, Utils.localHostName(), actor) + val future = trackerActor.ask(msg)(askTimeout) + Await.result(future, askTimeout) + } + + override protected def onReceiverStop(message: String, error: Option[Throwable]) { logInfo("Deregistering receiver " + streamId) val errorString = error.map(Throwables.getStackTraceAsString).getOrElse("") val future = trackerActor.ask( @@ -170,11 +181,6 @@ private[streaming] class ReceiverSupervisorImpl( logInfo("Stopped receiver " + streamId) } - override def stop(message: String, error: Option[Throwable]) { - super.stop(message, error) - env.actorSystem.stop(actor) - } - /** Generate new block ID */ private def nextBlockId = StreamBlockId(streamId, newBlockId.getAndIncrement) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala index 9c69a2a4e21f5..a68aecb881117 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala @@ -18,8 +18,10 @@ package org.apache.spark.streaming.scheduler import org.apache.spark.streaming.Time +import org.apache.spark.annotation.DeveloperApi /** + * :: DeveloperApi :: * Class having information on completed batches. * @param batchTime Time of the batch * @param submissionTime Clock time of when jobs of this batch was submitted to @@ -27,6 +29,7 @@ import org.apache.spark.streaming.Time * @param processingStartTime Clock time of when the first job of this batch started processing * @param processingEndTime Clock time of when the last job of this batch finished processing */ +@DeveloperApi case class BatchInfo( batchTime: Time, receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]], diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala new file mode 100644 index 0000000000000..d7e39c528c519 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.scheduler + +import akka.actor.ActorRef +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Class having information about a receiver + */ +@DeveloperApi +case class ReceiverInfo( + streamId: Int, + name: String, + private[streaming] val actor: ActorRef, + active: Boolean, + location: String, + lastErrorMessage: String = "", + lastError: String = "" + ) { +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 557e0961d5944..5307fe189d717 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -28,13 +28,8 @@ import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.receiver.{Receiver, ReceiverSupervisorImpl, StopReceiver} import org.apache.spark.util.AkkaUtils -/** Information about receiver */ -case class ReceiverInfo(streamId: Int, typ: String, location: String) { - override def toString = s"$typ-$streamId" -} - /** Information about blocks received by the receiver */ -case class ReceivedBlockInfo( +private[streaming] case class ReceivedBlockInfo( streamId: Int, blockId: StreamBlockId, numRecords: Long, @@ -69,7 +64,7 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { val receiverInputStreams = ssc.graph.getReceiverInputStreams() val receiverInputStreamMap = Map(receiverInputStreams.map(x => (x.id, x)): _*) val receiverExecutor = new ReceiverLauncher() - val receiverInfo = new HashMap[Int, ActorRef] with SynchronizedMap[Int, ActorRef] + val receiverInfo = new HashMap[Int, ReceiverInfo] with SynchronizedMap[Int, ReceiverInfo] val receivedBlockInfo = new HashMap[Int, SynchronizedQueue[ReceivedBlockInfo]] with SynchronizedMap[Int, SynchronizedQueue[ReceivedBlockInfo]] val timeout = AkkaUtils.askTimeout(ssc.conf) @@ -129,17 +124,23 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { if (!receiverInputStreamMap.contains(streamId)) { throw new Exception("Register received for unexpected id " + streamId) } - receiverInfo += ((streamId, receiverActor)) - ssc.scheduler.listenerBus.post(StreamingListenerReceiverStarted( - ReceiverInfo(streamId, typ, host) - )) + receiverInfo(streamId) = ReceiverInfo( + streamId, s"${typ}-${streamId}", receiverActor, true, host) + ssc.scheduler.listenerBus.post(StreamingListenerReceiverStarted(receiverInfo(streamId))) logInfo("Registered receiver for stream " + streamId + " from " + sender.path.address) } /** Deregister a receiver */ def deregisterReceiver(streamId: Int, message: String, error: String) { - receiverInfo -= streamId - ssc.scheduler.listenerBus.post(StreamingListenerReceiverStopped(streamId, message, error)) + val newReceiverInfo = receiverInfo.get(streamId) match { + case Some(oldInfo) => + oldInfo.copy(actor = null, active = false, lastErrorMessage = message, lastError = error) + case None => + logWarning("No prior receiver info") + ReceiverInfo(streamId, "", null, false, "", lastErrorMessage = message, lastError = error) + } + receiverInfo(streamId) = newReceiverInfo + ssc.scheduler.listenerBus.post(StreamingListenerReceiverStopped(receiverInfo(streamId))) val messageWithError = if (error != null && !error.isEmpty) { s"$message - $error" } else { @@ -157,7 +158,15 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { /** Report error sent by a receiver */ def reportError(streamId: Int, message: String, error: String) { - ssc.scheduler.listenerBus.post(StreamingListenerReceiverError(streamId, message, error)) + val newReceiverInfo = receiverInfo.get(streamId) match { + case Some(oldInfo) => + oldInfo.copy(lastErrorMessage = message, lastError = error) + case None => + logWarning("No prior receiver info") + ReceiverInfo(streamId, "", null, false, "", lastErrorMessage = message, lastError = error) + } + receiverInfo(streamId) = newReceiverInfo + ssc.scheduler.listenerBus.post(StreamingListenerReceiverError(receiverInfo(streamId))) val messageWithError = if (error != null && !error.isEmpty) { s"$message - $error" } else { @@ -270,7 +279,8 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { /** Stops the receivers. */ private def stopReceivers() { // Signal the receivers to stop - receiverInfo.values.foreach(_ ! StopReceiver) + receiverInfo.values.flatMap { info => Option(info.actor)} + .foreach { _ ! StopReceiver } logInfo("Sent stop signal to all " + receiverInfo.size + " receivers") } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala index 9d6ec1fa33549..ed1aa114e19d9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala @@ -20,28 +20,45 @@ package org.apache.spark.streaming.scheduler import scala.collection.mutable.Queue import org.apache.spark.util.Distribution +import org.apache.spark.annotation.DeveloperApi -/** Base trait for events related to StreamingListener */ +/** + * :: DeveloperApi :: + * Base trait for events related to StreamingListener + */ +@DeveloperApi sealed trait StreamingListenerEvent +@DeveloperApi case class StreamingListenerBatchSubmitted(batchInfo: BatchInfo) extends StreamingListenerEvent + +@DeveloperApi case class StreamingListenerBatchCompleted(batchInfo: BatchInfo) extends StreamingListenerEvent + +@DeveloperApi case class StreamingListenerBatchStarted(batchInfo: BatchInfo) extends StreamingListenerEvent +@DeveloperApi case class StreamingListenerReceiverStarted(receiverInfo: ReceiverInfo) extends StreamingListenerEvent -case class StreamingListenerReceiverError(streamId: Int, message: String, error: String) + +@DeveloperApi +case class StreamingListenerReceiverError(receiverInfo: ReceiverInfo) extends StreamingListenerEvent -case class StreamingListenerReceiverStopped(streamId: Int, message: String, error: String) + +@DeveloperApi +case class StreamingListenerReceiverStopped(receiverInfo: ReceiverInfo) extends StreamingListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ private[scheduler] case object StreamingListenerShutdown extends StreamingListenerEvent /** + * :: DeveloperApi :: * A listener interface for receiving information about an ongoing streaming * computation. */ +@DeveloperApi trait StreamingListener { /** Called when a receiver has been started */ @@ -65,9 +82,11 @@ trait StreamingListener { /** + * :: DeveloperApi :: * A simple StreamingListener that logs summary statistics across Spark Streaming batches * @param numBatchInfos Number of last batches to consider for generating statistics (default: 10) */ +@DeveloperApi class StatsReportListener(numBatchInfos: Int = 10) extends StreamingListener { // Queue containing latest completed batches val batchInfos = new Queue[BatchInfo]() diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index 14c33c728bfe1..f61069b56db5e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -23,9 +23,9 @@ import scala.collection.mutable.{Queue, HashMap} import org.apache.spark.streaming.scheduler.StreamingListenerReceiverStarted import org.apache.spark.streaming.scheduler.StreamingListenerBatchStarted import org.apache.spark.streaming.scheduler.BatchInfo -import org.apache.spark.streaming.scheduler.ReceiverInfo import org.apache.spark.streaming.scheduler.StreamingListenerBatchSubmitted import org.apache.spark.util.Distribution +import org.apache.spark.Logging private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) @@ -40,9 +40,21 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) val batchDuration = ssc.graph.batchDuration.milliseconds - override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) = { + override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { synchronized { - receiverInfos.put(receiverStarted.receiverInfo.streamId, receiverStarted.receiverInfo) + receiverInfos(receiverStarted.receiverInfo.streamId) = receiverStarted.receiverInfo + } + } + + override def onReceiverError(receiverError: StreamingListenerReceiverError) { + synchronized { + receiverInfos(receiverError.receiverInfo.streamId) = receiverError.receiverInfo + } + } + + override def onReceiverStopped(receiverStopped: StreamingListenerReceiverStopped) { + synchronized { + receiverInfos(receiverStopped.receiverInfo.streamId) = receiverStopped.receiverInfo } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 8fe1219356cdc..451b23e01c995 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -78,25 +78,33 @@ private[ui] class StreamingPage(parent: StreamingTab) val table = if (receivedRecordDistributions.size > 0) { val headerRow = Seq( "Receiver", + "Status", "Location", "Records in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]", "Minimum rate\n[records/sec]", - "25th percentile rate\n[records/sec]", "Median rate\n[records/sec]", - "75th percentile rate\n[records/sec]", - "Maximum rate\n[records/sec]" + "Maximum rate\n[records/sec]", + "Last Error" ) val dataRows = (0 until listener.numReceivers).map { receiverId => val receiverInfo = listener.receiverInfo(receiverId) - val receiverName = receiverInfo.map(_.toString).getOrElse(s"Receiver-$receiverId") + val receiverName = receiverInfo.map(_.name).getOrElse(s"Receiver-$receiverId") + val receiverActive = receiverInfo.map { info => + if (info.active) "ACTIVE" else "INACTIVE" + }.getOrElse(emptyCell) val receiverLocation = receiverInfo.map(_.location).getOrElse(emptyCell) val receiverLastBatchRecords = formatNumber(lastBatchReceivedRecord(receiverId)) val receivedRecordStats = receivedRecordDistributions(receiverId).map { d => - d.getQuantiles().map(r => formatNumber(r.toLong)) + d.getQuantiles(Seq(0.0, 0.5, 1.0)).map(r => formatNumber(r.toLong)) }.getOrElse { Seq(emptyCell, emptyCell, emptyCell, emptyCell, emptyCell) } - Seq(receiverName, receiverLocation, receiverLastBatchRecords) ++ receivedRecordStats + val receiverLastError = listener.receiverInfo(receiverId).map { info => + val msg = s"${info.lastErrorMessage} - ${info.lastError}" + if (msg.size > 100) msg.take(97) + "..." else msg + }.getOrElse(emptyCell) + Seq(receiverName, receiverActive, receiverLocation, receiverLastBatchRecords) ++ + receivedRecordStats ++ Seq(receiverLastError) } Some(listingTable(headerRow, dataRows)) } else { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala index ff3619a59042d..303d149d285e1 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala @@ -94,9 +94,13 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { // Verify restarting actually stops and starts the receiver receiver.restart("restarting", null, 100) - assert(receiver.isStopped) - assert(receiver.onStopCalled) + eventually(timeout(50 millis), interval(10 millis)) { + // receiver will be stopped async + assert(receiver.isStopped) + assert(receiver.onStopCalled) + } eventually(timeout(1000 millis), interval(100 millis)) { + // receiver will be started async assert(receiver.onStartCalled) assert(executor.isReceiverStarted) assert(receiver.isStarted) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 458dd3a2b13d8..ef0efa552ceaf 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -66,7 +66,7 @@ class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers { test("receiver info reporting") { val ssc = new StreamingContext("local[2]", "test", Milliseconds(1000)) - val inputStream = ssc.networkStream(new StreamingListenerSuiteReceiver) + val inputStream = ssc.receiverStream(new StreamingListenerSuiteReceiver) inputStream.foreachRDD(_.count) val collector = new ReceiverInfoCollector @@ -75,8 +75,8 @@ class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers { ssc.start() try { eventually(timeout(1000 millis), interval(20 millis)) { - collector.startedReceiverInfo should have size 1 - collector.startedReceiverInfo(0).streamId should equal (0) + collector.startedReceiverStreamIds.size should be >= 1 + collector.startedReceiverStreamIds(0) should equal (0) collector.stoppedReceiverStreamIds should have size 1 collector.stoppedReceiverStreamIds(0) should equal (0) collector.receiverErrors should have size 1 @@ -108,20 +108,21 @@ class BatchInfoCollector extends StreamingListener { /** Listener that collects information on processed batches */ class ReceiverInfoCollector extends StreamingListener { - val startedReceiverInfo = new ArrayBuffer[ReceiverInfo] + val startedReceiverStreamIds = new ArrayBuffer[Int] val stoppedReceiverStreamIds = new ArrayBuffer[Int]() val receiverErrors = new ArrayBuffer[(Int, String, String)]() override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { - startedReceiverInfo += receiverStarted.receiverInfo + startedReceiverStreamIds += receiverStarted.receiverInfo.streamId } override def onReceiverStopped(receiverStopped: StreamingListenerReceiverStopped) { - stoppedReceiverStreamIds += receiverStopped.streamId + stoppedReceiverStreamIds += receiverStopped.receiverInfo.streamId } override def onReceiverError(receiverError: StreamingListenerReceiverError) { - receiverErrors += ((receiverError.streamId, receiverError.message, receiverError.error)) + receiverErrors += ((receiverError.receiverInfo.streamId, + receiverError.receiverInfo.lastErrorMessage, receiverError.receiverInfo.lastError)) } } From 86ff8b10270bbe2579cdb1dc2297a9f4e145973e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 24 Apr 2014 21:42:33 -0700 Subject: [PATCH 172/641] Generalize pattern for planning hash joins. This will be helpful for [SPARK-1495](https://issues.apache.org/jira/browse/SPARK-1495) and other cases where we want to have custom hash join implementations but don't want to repeat the logic for finding the join keys. Author: Michael Armbrust Closes #418 from marmbrus/hashFilter and squashes the following commits: d5cc79b [Michael Armbrust] Address @rxin 's comments. 366b6d9 [Michael Armbrust] style fixes 14560eb [Michael Armbrust] Generalize pattern for planning hash joins. f4809c1 [Michael Armbrust] Move common functions to PredicateHelper. --- .../sql/catalyst/expressions/predicates.scala | 29 ++++++++--- .../sql/catalyst/planning/patterns.scala | 52 +++++++++++++++++++ .../spark/sql/execution/SparkStrategies.scala | 49 +++-------------- 3 files changed, 82 insertions(+), 48 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index da5b2cf5b0362..82c7af684459f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.trees -import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.analysis.UnresolvedException -import org.apache.spark.sql.catalyst.types.{BooleanType, StringType, TimestampType} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.catalyst.types.BooleanType + object InterpretedPredicate { def apply(expression: Expression): (Row => Boolean) = { @@ -37,10 +38,26 @@ trait Predicate extends Expression { } trait PredicateHelper { - def splitConjunctivePredicates(condition: Expression): Seq[Expression] = condition match { - case And(cond1, cond2) => splitConjunctivePredicates(cond1) ++ splitConjunctivePredicates(cond2) - case other => other :: Nil + protected def splitConjunctivePredicates(condition: Expression): Seq[Expression] = { + condition match { + case And(cond1, cond2) => + splitConjunctivePredicates(cond1) ++ splitConjunctivePredicates(cond2) + case other => other :: Nil + } } + + /** + * Returns true if `expr` can be evaluated using only the output of `plan`. This method + * can be used to determine when is is acceptable to move expression evaluation within a query + * plan. + * + * For example consider a join between two relations R(a, b) and S(c, d). + * + * `canEvaluate(Equals(a,b), R)` returns `true` where as `canEvaluate(Equals(a,c), R)` returns + * `false`. + */ + protected def canEvaluate(expr: Expression, plan: LogicalPlan): Boolean = + expr.references.subsetOf(plan.outputSet) } abstract class BinaryPredicate extends BinaryExpression with Predicate { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 6dd816aa91dd1..0e3a8a6bd30a8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -19,7 +19,10 @@ package org.apache.spark.sql.catalyst.planning import scala.annotation.tailrec +import org.apache.spark.sql.Logging + import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ /** @@ -101,6 +104,55 @@ object PhysicalOperation extends PredicateHelper { } } +/** + * A pattern that finds joins with equality conditions that can be evaluated using hashing + * techniques. For inner joins, any filters on top of the join operator are also matched. + */ +object HashFilteredJoin extends Logging with PredicateHelper { + /** (joinType, rightKeys, leftKeys, condition, leftChild, rightChild) */ + type ReturnType = + (JoinType, Seq[Expression], Seq[Expression], Option[Expression], LogicalPlan, LogicalPlan) + + def unapply(plan: LogicalPlan): Option[ReturnType] = plan match { + // All predicates can be evaluated for inner join (i.e., those that are in the ON + // clause and WHERE clause.) + case FilteredOperation(predicates, join @ Join(left, right, Inner, condition)) => + logger.debug(s"Considering hash inner join on: ${predicates ++ condition}") + splitPredicates(predicates ++ condition, join) + case join @ Join(left, right, joinType, condition) => + logger.debug(s"Considering hash join on: $condition") + splitPredicates(condition.toSeq, join) + case _ => None + } + + // Find equi-join predicates that can be evaluated before the join, and thus can be used + // as join keys. + def splitPredicates(allPredicates: Seq[Expression], join: Join): Option[ReturnType] = { + val Join(left, right, joinType, _) = join + val (joinPredicates, otherPredicates) = allPredicates.partition { + case Equals(l, r) if (canEvaluate(l, left) && canEvaluate(r, right)) || + (canEvaluate(l, right) && canEvaluate(r, left)) => true + case _ => false + } + + val joinKeys = joinPredicates.map { + case Equals(l, r) if canEvaluate(l, left) && canEvaluate(r, right) => (l, r) + case Equals(l, r) if canEvaluate(l, right) && canEvaluate(r, left) => (r, l) + } + + // Do not consider this strategy if there are no join keys. + if (joinKeys.nonEmpty) { + val leftKeys = joinKeys.map(_._1) + val rightKeys = joinKeys.map(_._2) + + Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right)) + } else { + logger.debug(s"Avoiding hash join with no join keys.") + None + } + } +} + /** * A pattern that collects all adjacent unions and returns their children as a Seq. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 500fde19719af..f763106da4e0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -28,51 +28,16 @@ import org.apache.spark.sql.parquet._ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SQLContext#SparkPlanner => - object HashJoin extends Strategy { + object HashJoin extends Strategy with PredicateHelper { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case FilteredOperation(predicates, logical.Join(left, right, Inner, condition)) => - logger.debug(s"Considering join: ${predicates ++ condition}") - // Find equi-join predicates that can be evaluated before the join, and thus can be used - // as join keys. Note we can only mix in the conditions with other predicates because the - // match above ensures that this is and Inner join. - val (joinPredicates, otherPredicates) = (predicates ++ condition).partition { - case Equals(l, r) if (canEvaluate(l, left) && canEvaluate(r, right)) || - (canEvaluate(l, right) && canEvaluate(r, left)) => true - case _ => false - } - - val joinKeys = joinPredicates.map { - case Equals(l,r) if canEvaluate(l, left) && canEvaluate(r, right) => (l, r) - case Equals(l,r) if canEvaluate(l, right) && canEvaluate(r, left) => (r, l) - } - - // Do not consider this strategy if there are no join keys. - if (joinKeys.nonEmpty) { - val leftKeys = joinKeys.map(_._1) - val rightKeys = joinKeys.map(_._2) - - val joinOp = execution.HashJoin( - leftKeys, rightKeys, BuildRight, planLater(left), planLater(right)) - - // Make sure other conditions are met if present. - if (otherPredicates.nonEmpty) { - execution.Filter(combineConjunctivePredicates(otherPredicates), joinOp) :: Nil - } else { - joinOp :: Nil - } - } else { - logger.debug(s"Avoiding spark join with no join keys.") - Nil - } + // Find inner joins where at least some predicates can be evaluated by matching hash keys + // using the HashFilteredJoin pattern. + case HashFilteredJoin(Inner, leftKeys, rightKeys, condition, left, right) => + val hashJoin = + execution.HashJoin(leftKeys, rightKeys, BuildRight, planLater(left), planLater(right)) + condition.map(Filter(_, hashJoin)).getOrElse(hashJoin) :: Nil case _ => Nil } - - private def combineConjunctivePredicates(predicates: Seq[Expression]) = - predicates.reduceLeft(And) - - /** Returns true if `expr` can be evaluated using only the output of `plan`. */ - protected def canEvaluate(expr: Expression, plan: LogicalPlan): Boolean = - expr.references subsetOf plan.outputSet } object PartialAggregation extends Strategy { From 06e82d94b6eba0b174b54c58280b3daca2f26dd8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 24 Apr 2014 21:51:17 -0700 Subject: [PATCH 173/641] [Typo] In the maven docs: chd -> cdh Author: Andrew Or Closes #548 from andrewor14/doc-typo and squashes the following commits: 3eaf4c4 [Andrew Or] chd -> cdh --- docs/building-with-maven.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 9cebaf12283fc..771d1933a13ea 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -45,7 +45,7 @@ For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with $ mvn -Pyarn-alpha -Dhadoop.version=2.0.5-alpha -Dyarn.version=2.0.5-alpha -DskipTests clean package # Cloudera CDH 4.2.0 with MapReduce v2 - $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package + $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-cdh4.2.0 -DskipTests clean package # Apache Hadoop 2.2.X ( e.g. 2.2.0 as below ) and newer $ mvn -Pyarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -DskipTests clean package From 095b5182536a43e2ae738be93294ee5215d86581 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Thu, 24 Apr 2014 21:51:52 -0700 Subject: [PATCH 174/641] Fix [SPARK-1078]: Remove the Unnecessary lift-json dependency Remove the Unnecessary lift-json dependency from pom.xml Author: Sandeep Closes #536 from techaddict/FIX-SPARK-1078 and squashes the following commits: bd0fd1d [Sandeep] Fix [SPARK-1078]: Replace lift-json with json4s-jackson. Remove the Unnecessary lift-json dependency from pom.xml --- pom.xml | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/pom.xml b/pom.xml index 4ff18afa227be..e292775c6267e 100644 --- a/pom.xml +++ b/pom.xml @@ -380,20 +380,6 @@ derby 10.4.2.0 - - net.liftweb - lift-json_${scala.binary.version} - 2.5.1 - - - - org.scala-lang - scalap - - - com.codahale.metrics metrics-core @@ -851,7 +837,7 @@ - + hadoop-provided @@ -896,6 +882,6 @@ - + From e03bc379ee03fde0ee4fa578d3c39aae35c63f01 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 24 Apr 2014 23:07:54 -0700 Subject: [PATCH 175/641] SPARK-1242 Add aggregate to python rdd Author: Holden Karau Closes #139 from holdenk/add_aggregate_to_python_api and squashes the following commits: 0f39ae3 [Holden Karau] Merge in master 4879c75 [Holden Karau] CR feedback, fix issue with empty RDDs in aggregate 70b4724 [Holden Karau] Style fixes from code review 96b047b [Holden Karau] Add aggregate to python rdd --- python/pyspark/rdd.py | 31 +++++++++++++++++++++++++++++-- 1 file changed, 29 insertions(+), 2 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index d73ab7006e9c7..a59778c72130e 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -599,7 +599,7 @@ def _collect_iterator_through_file(self, iterator): def reduce(self, f): """ Reduces the elements of this RDD using the specified commutative and - associative binary operator. + associative binary operator. Currently reduces partitions locally. >>> from operator import add >>> sc.parallelize([1, 2, 3, 4, 5]).reduce(add) @@ -641,7 +641,34 @@ def func(iterator): vals = self.mapPartitions(func).collect() return reduce(op, vals, zeroValue) - # TODO: aggregate + def aggregate(self, zeroValue, seqOp, combOp): + """ + Aggregate the elements of each partition, and then the results for all + the partitions, using a given combine functions and a neutral "zero + value." + + The functions C{op(t1, t2)} is allowed to modify C{t1} and return it + as its result value to avoid object allocation; however, it should not + modify C{t2}. + + The first function (seqOp) can return a different result type, U, than + the type of this RDD. Thus, we need one operation for merging a T into an U + and one operation for merging two U + + >>> seqOp = (lambda x, y: (x[0] + y, x[1] + 1)) + >>> combOp = (lambda x, y: (x[0] + y[0], x[1] + y[1])) + >>> sc.parallelize([1, 2, 3, 4]).aggregate((0, 0), seqOp, combOp) + (10, 4) + >>> sc.parallelize([]).aggregate((0, 0), seqOp, combOp) + (0, 0) + """ + def func(iterator): + acc = zeroValue + for obj in iterator: + acc = seqOp(acc, obj) + yield acc + + return self.mapPartitions(func).fold(zeroValue, combOp) def max(self): From 45ad7f0ca7be4a89c066b246b0fee0c0c2e7c759 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 24 Apr 2014 23:22:03 -0700 Subject: [PATCH 176/641] Call correct stop(). Oopsie in #504. Author: Aaron Davidson Closes #527 from aarondav/stop and squashes the following commits: 8d1446a [Aaron Davidson] Call correct stop(). --- .../main/scala/org/apache/spark/storage/DiskBlockManager.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 054f66a8b7260..cf6ef0029a861 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -150,11 +150,12 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { override def run() { logDebug("Shutdown hook called") - stop() + DiskBlockManager.this.stop() } }) } + /** Cleanup local dirs and stop shuffle sender. */ private[spark] def stop() { localDirs.foreach { localDir => if (localDir.isDirectory() && localDir.exists()) { From 6e101f1183f92769779bc8ac14813c063bf1ff3f Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 24 Apr 2014 23:34:00 -0700 Subject: [PATCH 177/641] SPARK-1607. Replace octal literals, removed in Scala 2.11, with hex literals Octal literals like "0700" are deprecated in Scala 2.10, generating a warning. They have been removed entirely in 2.11. See https://issues.scala-lang.org/browse/SI-7618 This change simply replaces two uses of octals with hex literals, which seemed the next-best representation since they express a bit mask (file permission in particular) Author: Sean Owen Closes #529 from srowen/SPARK-1607 and squashes the following commits: 1ee0e67 [Sean Owen] Use Integer.parseInt(...,8) for octal literal instead of hex equivalent 0102f3d [Sean Owen] Replace octal literals, removed in Scala 2.11, with hex literals --- .../scala/org/apache/spark/deploy/yarn/ClientBase.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 b403292d9cb2e..eb95d784318ea 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 @@ -59,9 +59,11 @@ trait ClientBase extends Logging { private val distCacheMgr = new ClientDistributedCacheManager() // Staging directory is private! -> rwx-------- - val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700: Short) + val STAGING_DIR_PERMISSION: FsPermission = + FsPermission.createImmutable(Integer.parseInt("700", 8): Short) // App files are world-wide readable and owner writable -> rw-r--r-- - val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644: Short) + val APP_FILE_PERMISSION: FsPermission = + FsPermission.createImmutable(Integer.parseInt("644", 8): Short) // TODO(harvey): This could just go in ClientArguments. def validateArgs() = { From dc3b640a0ab3501b678b591be3e99fbcf3badbec Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 24 Apr 2014 23:59:16 -0700 Subject: [PATCH 178/641] SPARK-1619 Launch spark-shell with spark-submit This simplifies the shell a bunch and passes all arguments through to spark-submit. There is a tiny incompatibility from 0.9.1 which is that you can't put `-c` _or_ `--cores`, only `--cores`. However, spark-submit will give a good error message in this case, I don't think many people used this, and it's a trivial change for users. Author: Patrick Wendell Closes #542 from pwendell/spark-shell and squashes the following commits: 9eb3e6f [Patrick Wendell] Updating Spark docs b552459 [Patrick Wendell] Andrew's feedback 97720fa [Patrick Wendell] Review feedback aa2900b [Patrick Wendell] SPARK-1619 Launch spark-shell with spark-submit --- .gitignore | 1 + bin/spark-shell | 177 +----------------- bin/spark-submit | 10 +- .../org/apache/spark/deploy/SparkSubmit.scala | 10 +- .../spark/deploy/SparkSubmitArguments.scala | 2 +- docs/scala-programming-guide.md | 9 +- docs/spark-debugger.md | 2 +- docs/spark-standalone.md | 4 +- docs/streaming-programming-guide.md | 6 +- make-distribution.sh | 2 +- .../org/apache/spark/repl/SparkILoop.scala | 5 +- 11 files changed, 39 insertions(+), 189 deletions(-) diff --git a/.gitignore b/.gitignore index cd9f90d55932c..857e9feb953bd 100644 --- a/.gitignore +++ b/.gitignore @@ -18,6 +18,7 @@ conf/java-opts conf/spark-env.sh conf/streaming-env.sh conf/log4j.properties +conf/spark-defaults.conf docs/_site docs/api target/ diff --git a/bin/spark-shell b/bin/spark-shell index ea12d256b23a1..f1f3c18877ed4 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -19,9 +19,8 @@ # # Shell script for starting the Spark Shell REPL -# Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT} -# if those two env vars are set in spark-env.sh but MASTER is not. +args="$@" cygwin=false case "`uname`" in CYGWIN*) cygwin=true;; @@ -30,133 +29,16 @@ esac # Enter posix mode for bash set -o posix +if [[ "$@" == *--help* ]]; then + echo "Usage: ./bin/spark-shell [options]" + ./bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 +fi + ## Global script variables FWDIR="$(cd `dirname $0`/..; pwd)" -SPARK_REPL_OPTS="${SPARK_REPL_OPTS:-""}" -DEFAULT_MASTER="local[*]" -MASTER=${MASTER:-""} - -info_log=0 - -#CLI Color Templates -txtund=$(tput sgr 0 1) # Underline -txtbld=$(tput bold) # Bold -bldred=${txtbld}$(tput setaf 1) # red -bldyel=${txtbld}$(tput setaf 3) # yellow -bldblu=${txtbld}$(tput setaf 4) # blue -bldwht=${txtbld}$(tput setaf 7) # white -txtrst=$(tput sgr0) # Reset -info=${bldwht}*${txtrst} # Feedback -pass=${bldblu}*${txtrst} -warn=${bldred}*${txtrst} -ques=${bldblu}?${txtrst} - -# Helper function to describe the script usage -function usage() { - cat << EOF -${txtbld}Usage${txtrst}: spark-shell [OPTIONS] - -${txtbld}OPTIONS${txtrst}: - -h --help : Print this help information. - -c --cores : The maximum number of cores to be used by the Spark Shell. - -em --executor-memory : The memory used by each executor of the Spark Shell, the number - is followed by m for megabytes or g for gigabytes, e.g. "1g". - -dm --driver-memory : The memory used by the Spark Shell, the number is followed - by m for megabytes or g for gigabytes, e.g. "1g". - -m --master : A full string that describes the Spark Master, defaults to "local[*]" - e.g. "spark://localhost:7077". - --log-conf : Enables logging of the supplied SparkConf as INFO at start of the - Spark Context. - -e.g. - spark-shell -m spark://localhost:7077 -c 4 -dm 512m -em 2g - -EOF -} - -function out_error(){ - echo -e "${txtund}${bldred}ERROR${txtrst}: $1" - usage - exit 1 -} - -function log_info(){ - [ $info_log -eq 1 ] && echo -e "${bldyel}INFO${txtrst}: $1" -} - -function log_warn(){ - echo -e "${txtund}${bldyel}WARN${txtrst}: $1" -} - -# PATTERNS used to validate more than one optional arg. -ARG_FLAG_PATTERN="^-" -MEM_PATTERN="^[0-9]+[m|g|M|G]$" -NUM_PATTERN="^[0-9]+$" -PORT_PATTERN="^[0-9]+$" - -# Setters for optional args. -function set_cores(){ - CORE_PATTERN="^[0-9]+$" - if [[ "$1" =~ $CORE_PATTERN ]]; then - SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.cores.max=$1" - else - out_error "wrong format for $2" - fi -} - -function set_em(){ - if [[ $1 =~ $MEM_PATTERN ]]; then - SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.executor.memory=$1" - else - out_error "wrong format for $2" - fi -} - -function set_dm(){ - if [[ $1 =~ $MEM_PATTERN ]]; then - export SPARK_DRIVER_MEMORY=$1 - else - out_error "wrong format for $2" - fi -} - -function set_spark_log_conf(){ - SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.logConf=$1" -} - -function set_spark_master(){ - if ! [[ "$1" =~ $ARG_FLAG_PATTERN ]]; then - export MASTER="$1" - else - out_error "wrong format for $2" - fi -} - -function resolve_spark_master(){ - # Set MASTER from spark-env if possible - DEFAULT_SPARK_MASTER_PORT=7077 - if [ -z "$MASTER" ]; then - . $FWDIR/bin/load-spark-env.sh - if [ -n "$SPARK_MASTER_IP" ]; then - SPARK_MASTER_PORT="${SPARK_MASTER_PORT:-"$DEFAULT_SPARK_MASTER_PORT"}" - export MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" - fi - fi - - if [ -z "$MASTER" ]; then - export MASTER="$DEFAULT_MASTER" - fi - -} - function main(){ - log_info "Base Directory set to $FWDIR" - - resolve_spark_master - log_info "Spark Master is $MASTER" - - log_info "Spark REPL options $SPARK_REPL_OPTS" if $cygwin; then # Workaround for issue involving JLine and Cygwin # (see http://sourceforge.net/p/jline/bugs/40/). @@ -165,55 +47,14 @@ function main(){ # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 export SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" + $FWDIR/bin/spark-submit spark-internal "$args" --class org.apache.spark.repl.Main stty icanon echo > /dev/null 2>&1 else export SPARK_REPL_OPTS - $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" + $FWDIR/bin/spark-submit spark-internal "$args" --class org.apache.spark.repl.Main fi } -for option in "$@" -do - case $option in - -h | --help ) - usage - exit 1 - ;; - -c | --cores) - shift - _1=$1 - shift - set_cores $_1 "-c/--cores" - ;; - -em | --executor-memory) - shift - _1=$1 - shift - set_em $_1 "-em/--executor-memory" - ;; - -dm | --driver-memory) - shift - _1=$1 - shift - set_dm $_1 "-dm/--driver-memory" - ;; - -m | --master) - shift - _1=$1 - shift - set_spark_master $_1 "-m/--master" - ;; - --log-conf) - shift - set_spark_log_conf "true" - info_log=1 - ;; - ?) - ;; - esac -done - # Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in # binary distribution of Spark where Scala is not installed exit_status=127 diff --git a/bin/spark-submit b/bin/spark-submit index 498d0b27bacdf..b2a1dca721dff 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -21,15 +21,15 @@ export SPARK_HOME="$(cd `dirname $0`/..; pwd)" ORIG_ARGS=$@ while (($#)); do - if [ $1 = "--deploy-mode" ]; then + if [ "$1" = "--deploy-mode" ]; then DEPLOY_MODE=$2 - elif [ $1 = "--driver-memory" ]; then + elif [ "$1" = "--driver-memory" ]; then DRIVER_MEMORY=$2 - elif [ $1 = "--driver-library-path" ]; then + elif [ "$1" = "--driver-library-path" ]; then export _SPARK_LIBRARY_PATH=$2 - elif [ $1 = "--driver-class-path" ]; then + elif [ "$1" = "--driver-class-path" ]; then export SPARK_CLASSPATH="$SPARK_CLASSPATH:$2" - elif [ $1 = "--driver-java-options" ]; then + elif [ "$1" = "--driver-java-options" ]; then export SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $2" fi shift diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index a2efddbfe3f69..58aa6d951a204 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -38,6 +38,12 @@ object SparkSubmit { private var clusterManager: Int = LOCAL + /** + * A special jar name that indicates the class being run is inside of Spark itself, + * and therefore no user jar is needed. + */ + private val RESERVED_JAR_NAME = "spark-internal" + def main(args: Array[String]) { val appArgs = new SparkSubmitArguments(args) if (appArgs.verbose) { @@ -113,7 +119,9 @@ object SparkSubmit { if (!deployOnCluster) { childMainClass = appArgs.mainClass - childClasspath += appArgs.primaryResource + if (appArgs.primaryResource != RESERVED_JAR_NAME) { + childClasspath += appArgs.primaryResource + } } else if (clusterManager == YARN) { childMainClass = "org.apache.spark.deploy.yarn.Client" childArgs += ("--jar", appArgs.primaryResource) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index c3e8c6b8c65a9..c545b093ac82e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -107,7 +107,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { deployMode = Option(deployMode).getOrElse(System.getenv("DEPLOY_MODE")) // Global defaults. These should be keep to minimum to avoid confusing behavior. - master = Option(master).getOrElse("local") + master = Option(master).getOrElse("local[*]") } /** Ensure that required fields exists. Call this only once all defaults are loaded. */ diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index a3171709ff05b..b8d89cf00ffbd 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -60,17 +60,18 @@ which avoids hard-coding the master name in your application. In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the -context connects to using the `MASTER` environment variable, and you can add JARs to the classpath -with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on exactly four cores, use +context connects to using the `--master` argument, and you can add JARs to the classpath +by passing a comma separated list to the `--jars` argument. For example, to run +`bin/spark-shell` on exactly four cores, use {% highlight bash %} -$ MASTER=local[4] ./bin/spark-shell +$ ./bin/spark-shell --master local[4] {% endhighlight %} Or, to also add `code.jar` to its classpath, use: {% highlight bash %} -$ MASTER=local[4] ADD_JARS=code.jar ./bin/spark-shell +$ ./bin/spark-shell --master local[4] --jars code.jar {% endhighlight %} ### Master URLs diff --git a/docs/spark-debugger.md b/docs/spark-debugger.md index 891c2bfa8943d..35d06c51aaf0e 100644 --- a/docs/spark-debugger.md +++ b/docs/spark-debugger.md @@ -39,7 +39,7 @@ where `path/to/event-log` is where you want the event log to go relative to `$SP ### Loading the event log into the debugger -1. Run a Spark shell with `MASTER=host ./bin/spark-shell`. +1. Run a Spark shell with `./bin/spark-shell --master hist`. 2. Use `EventLogReader` to load the event log as follows: {% highlight scala %} spark> val r = new spark.EventLogReader(sc, Some("path/to/event-log")) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 7e4eea323aa63..dc7f206e03996 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -139,12 +139,12 @@ constructor](scala-programming-guide.html#initializing-spark). To run an interactive Spark shell against the cluster, run the following command: - MASTER=spark://IP:PORT ./bin/spark-shell + ./bin/spark-shell --master spark://IP:PORT Note that if you are running spark-shell from one of the spark cluster machines, the `bin/spark-shell` script will automatically set MASTER from the `SPARK_MASTER_IP` and `SPARK_MASTER_PORT` variables in `conf/spark-env.sh`. -You can also pass an option `-c ` to control the number of cores that spark-shell uses on the cluster. +You can also pass an option `--cores ` to control the number of cores that spark-shell uses on the cluster. # Launching Compiled Spark Applications diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 946d6c4879cf3..7ad06427cac5c 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -272,12 +272,10 @@ Time: 1357008430000 ms -If you plan to run the Scala code for Spark Streaming-based use cases in the Spark -shell, you should start the shell with the SparkConfiguration pre-configured to -discard old batches periodically: +You can also use Spark Streaming directly from the Spark shell: {% highlight bash %} -$ SPARK_JAVA_OPTS=-Dspark.cleaner.ttl=10000 bin/spark-shell +$ bin/spark-shell {% endhighlight %} ... and create your StreamingContext by wrapping the existing interactive shell diff --git a/make-distribution.sh b/make-distribution.sh index 62a28217740cd..55fe6c09d0d66 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -36,7 +36,7 @@ # 2) cd to deploy dir; ./sbin/start-master.sh # 3) Verify master is up by visiting web page, ie http://master-ip:8080. Note the spark:// URL. # 4) ./sbin/start-slave.sh 1 <> -# 5) MASTER="spark://my-master-ip:7077" ./bin/spark-shell +# 5) ./bin/spark-shell --master spark://my-master-ip:7077 # # Figure out where the Spark framework is installed diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index beb40e87024bd..296da740687ec 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -963,8 +963,9 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, val master = this.master match { case Some(m) => m case None => { - val prop = System.getenv("MASTER") - if (prop != null) prop else "local[*]" + val envMaster = sys.env.get("MASTER") + val propMaster = sys.props.get("spark.master") + envMaster.orElse(propMaster).getOrElse("local[*]") } } master From a24d918c71f6ac4adbe3ae363ef69f4658118938 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 25 Apr 2014 11:12:41 -0700 Subject: [PATCH 179/641] SPARK-1621 Upgrade Chill to 0.3.6 It registers more Scala classes, including things like Ranges that we had to register manually before. See https://github.com/twitter/chill/releases for Chill's change log. Author: Matei Zaharia Closes #543 from mateiz/chill-0.3.6 and squashes the following commits: a1dc5e0 [Matei Zaharia] Upgrade Chill to 0.3.6 and remove our special registration of Ranges --- .../org/apache/spark/serializer/KryoSerializer.scala | 12 +++++------- pom.xml | 2 +- project/SparkBuild.scala | 6 +++--- 3 files changed, 9 insertions(+), 11 deletions(-) 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 d1e8c3ef63622..c4daec7875d26 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -58,7 +58,9 @@ class KryoSerializer(conf: SparkConf) // Do this before we invoke the user registrator so the user registrator can override this. kryo.setReferences(referenceTracking) - for (cls <- KryoSerializer.toRegister) kryo.register(cls) + for (cls <- KryoSerializer.toRegister) { + kryo.register(cls) + } // Allow sending SerializableWritable kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) @@ -77,7 +79,7 @@ class KryoSerializer(conf: SparkConf) } // Register Chill's classes; we do this after our ranges and the user's own classes to let - // our code override the generic serialziers in Chill for things like Seq + // our code override the generic serializers in Chill for things like Seq new AllScalaRegistrar().apply(kryo) kryo.setClassLoader(classLoader) @@ -176,10 +178,6 @@ private[serializer] object KryoSerializer { classOf[GetBlock], classOf[MapStatus], classOf[BlockManagerId], - classOf[Array[Byte]], - (1 to 10).getClass, - (1 until 10).getClass, - (1L to 10L).getClass, - (1L until 10L).getClass + classOf[Array[Byte]] ) } diff --git a/pom.xml b/pom.xml index e292775c6267e..e9117998252fc 100644 --- a/pom.xml +++ b/pom.xml @@ -125,7 +125,7 @@ 1.3.2 1.2.3 8.1.14.v20131031 - 0.3.1 + 0.3.6 3.0.0 64m diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 9cbc188c4d485..48f234ce87a44 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -294,7 +294,7 @@ object SparkBuild extends Build { ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings ++ ScalaStyleSettings ++ genjavadocSettings val akkaVersion = "2.2.3-shaded-protobuf" - val chillVersion = "0.3.1" + val chillVersion = "0.3.6" val codahaleMetricsVersion = "3.0.0" val jblasVersion = "1.2.3" val jettyVersion = "8.1.14.v20131031" @@ -412,8 +412,8 @@ object SparkBuild extends Build { ) def examplesSettings = sharedSettings ++ Seq( - name := "spark-examples", - jarName in assembly <<= version map { + name := "spark-examples", + jarName in assembly <<= version map { v => "spark-examples-" + v + "-hadoop" + hadoopVersion + ".jar" }, libraryDependencies ++= Seq( "com.twitter" %% "algebird-core" % "0.1.11", From 25a276dd21457fa0db76b6be93765682a50b784a Mon Sep 17 00:00:00 2001 From: WangTao Date: Fri, 25 Apr 2014 11:47:01 -0700 Subject: [PATCH 180/641] Delete the val that never used It seems that the val "startTime" and "endTime" is never used, so delete them. Author: WangTao Closes #553 from WangTaoTheTonic/master and squashes the following commits: 4fcb639 [WangTao] Delete the val that never used --- .../main/scala/org/apache/spark/storage/BlockMessage.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala index 337b45b727dec..a2bfce7b4a0fa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala @@ -57,7 +57,6 @@ private[spark] class BlockMessage() { } def set(buffer: ByteBuffer) { - val startTime = System.currentTimeMillis /* println() println("BlockMessage: ") @@ -100,7 +99,6 @@ private[spark] class BlockMessage() { data.flip() } - val finishTime = System.currentTimeMillis } def set(bufferMsg: BufferMessage) { @@ -115,7 +113,6 @@ private[spark] class BlockMessage() { def getLevel: StorageLevel = level def toBufferMessage: BufferMessage = { - val startTime = System.currentTimeMillis val buffers = new ArrayBuffer[ByteBuffer]() var buffer = ByteBuffer.allocate(4 + 4 + id.name.length * 2) buffer.putInt(typ).putInt(id.name.length) @@ -153,7 +150,6 @@ private[spark] class BlockMessage() { println() println() */ - val finishTime = System.currentTimeMillis Message.createBufferMessage(buffers) } From 8aaef5c75624b48806a05943454f7b72f3ca1bdc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?baishuo=28=E7=99=BD=E7=A1=95=29?= Date: Fri, 25 Apr 2014 13:18:49 -0700 Subject: [PATCH 181/641] Update KafkaWordCount.scala MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit modify the required args number Author: baishuo(白硕) Closes #523 from baishuo/master and squashes the following commits: 0368ba9 [baishuo(白硕)] Update KafkaWordCount.scala --- .../org/apache/spark/streaming/examples/KafkaWordCount.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index 3a18bd02b89ac..2aa4f1474a59e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -71,7 +71,7 @@ object KafkaWordCount { object KafkaWordCountProducer { def main(args: Array[String]) { - if (args.length < 2) { + if (args.length < 4) { System.err.println("Usage: KafkaWordCountProducer " + " ") System.exit(1) From df6d81425bf3b8830988288069f6863de873aee2 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 25 Apr 2014 14:17:38 -0700 Subject: [PATCH 182/641] SPARK-1607. HOTFIX: Fix syntax adapting Int result to Short Sorry folks. This should make the change for SPARK-1607 compile again. Verified this time with the yarn build enabled. Author: Sean Owen Closes #556 from srowen/SPARK-1607.2 and squashes the following commits: e3fe7a3 [Sean Owen] Fix syntax adapting Int result to Short --- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 eb95d784318ea..f2be8217a2f8a 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 @@ -60,10 +60,10 @@ trait ClientBase extends Logging { // Staging directory is private! -> rwx-------- val STAGING_DIR_PERMISSION: FsPermission = - FsPermission.createImmutable(Integer.parseInt("700", 8): Short) + FsPermission.createImmutable(Integer.parseInt("700", 8).toShort) // App files are world-wide readable and owner writable -> rw-r--r-- val APP_FILE_PERMISSION: FsPermission = - FsPermission.createImmutable(Integer.parseInt("644", 8): Short) + FsPermission.createImmutable(Integer.parseInt("644", 8).toShort) // TODO(harvey): This could just go in ClientArguments. def validateArgs() = { From 027f1b85f961ce16ee069afe3d90a36dce009994 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Fri, 25 Apr 2014 16:04:48 -0700 Subject: [PATCH 183/641] SPARK-1235: manage the DAGScheduler EventProcessActor with supervisor and refactor the DAGScheduler with Akka https://spark-project.atlassian.net/browse/SPARK-1235 In the current implementation, the running job will hang if the DAGScheduler crashes for some reason (eventProcessActor throws exception in receive() ) The reason is that the actor will automatically restart when the exception is thrown during the running but is not captured properly (Akka behaviour), and the JobWaiters are still waiting there for the completion of the tasks In this patch, I refactored the DAGScheduler with Akka and manage the eventProcessActor with supervisor, so that upon the failure of a eventProcessActor, the supervisor will terminate the EventProcessActor and close the SparkContext thanks for @kayousterhout and @markhamstra to give the hints in JIRA Author: CodingCat Author: Xiangrui Meng Author: Nan Zhu Closes #186 from CodingCat/SPARK-1235 and squashes the following commits: a7fb0ee [CodingCat] throw Exception on failure of creating DAG 124d82d [CodingCat] blocking the constructor until event actor is ready baf2d38 [CodingCat] fix the issue brought by non-blocking actorOf 35c886a [CodingCat] fix bug 82d08b3 [CodingCat] calling actorOf on system to ensure it is blocking 310a579 [CodingCat] style fix cd02d9a [Nan Zhu] small fix 561cfbc [CodingCat] recover doCheckpoint c048d0e [CodingCat] call submitWaitingStages for every event a9eea039 [CodingCat] address Matei's comments ac878ab [CodingCat] typo fix 5d1636a [CodingCat] re-trigger the test..... 9dfb033 [CodingCat] remove unnecessary changes a7a2a97 [CodingCat] add StageCancelled message fdf3b17 [CodingCat] just to retrigger the test...... 089bc2f [CodingCat] address andrew's comments 228f4b0 [CodingCat] address comments from Mark b68c1c7 [CodingCat] refactor DAGScheduler with Akka 810efd8 [Xiangrui Meng] akka solution --- .../scala/org/apache/spark/SparkContext.scala | 20 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 6 +- .../apache/spark/scheduler/DAGScheduler.scala | 419 ++++++++++-------- .../spark/scheduler/DAGSchedulerEvent.scala | 4 +- .../spark/scheduler/TaskSetManager.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 58 ++- .../scheduler/TaskSchedulerImplSuite.scala | 2 +- 7 files changed, 290 insertions(+), 221 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e9d2f5757963d..eb14d87467af7 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -300,10 +300,17 @@ class SparkContext(config: SparkConf) extends Logging { // Create and start the scheduler private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) - taskScheduler.start() + @volatile private[spark] var dagScheduler: DAGScheduler = _ + try { + dagScheduler = new DAGScheduler(this) + } catch { + case e: Exception => throw + new SparkException("DAGScheduler cannot be initialized due to %s".format(e.getMessage)) + } - @volatile private[spark] var dagScheduler = new DAGScheduler(this) - dagScheduler.start() + // start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's + // constructor + taskScheduler.start() private[spark] val cleaner: Option[ContextCleaner] = { if (conf.getBoolean("spark.cleaner.referenceTracking", true)) { @@ -1022,8 +1029,8 @@ class SparkContext(config: SparkConf) extends Logging { partitions: Seq[Int], allowLocal: Boolean, resultHandler: (Int, U) => Unit) { - partitions.foreach{ p => - require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p") + if (dagScheduler == null) { + throw new SparkException("SparkContext has been shutdown") } val callSite = getCallSite val cleanedFunc = clean(func) @@ -1132,9 +1139,6 @@ class SparkContext(config: SparkConf) extends Logging { 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/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index e8bbfbf01679c..3b3524f33e811 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1142,9 +1142,9 @@ abstract class RDD[T: ClassTag]( @transient private var doCheckpointCalled = false /** - * Performs the checkpointing of this RDD by saving this. It is called by the DAGScheduler - * after a job using this RDD has completed (therefore the RDD has been materialized and - * potentially stored in memory). doCheckpoint() is called recursively on the parent RDDs. + * Performs the checkpointing of this RDD by saving this. It is called after a job using this RDD + * has completed (therefore the RDD has been materialized and potentially stored in memory). + * doCheckpoint() is called recursively on the parent RDDs. */ private[spark] def doCheckpoint() { if (!doCheckpointCalled) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index dbde9b591dccc..ff411e24a3d85 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -22,10 +22,16 @@ import java.util.Properties import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} +import scala.concurrent.Await import scala.concurrent.duration._ +import scala.language.postfixOps import scala.reflect.ClassTag import akka.actor._ +import akka.actor.OneForOneStrategy +import akka.actor.SupervisorStrategy.Stop +import akka.pattern.ask +import akka.util.Timeout import org.apache.spark._ import org.apache.spark.executor.TaskMetrics @@ -47,14 +53,11 @@ import org.apache.spark.util.Utils * not caused by shuffle file loss are handled by the TaskScheduler, which will retry each task * a small number of times before cancelling the whole stage. * - * THREADING: This class runs all its logic in a single thread executing the run() method, to which - * events are submitted using a synchronized queue (eventQueue). The public API methods, such as - * runJob, taskEnded and executorLost, post events asynchronously to this queue. All other methods - * should be private. */ private[spark] class DAGScheduler( - taskScheduler: TaskScheduler, + private[scheduler] val sc: SparkContext, + private[scheduler] val taskScheduler: TaskScheduler, listenerBus: LiveListenerBus, mapOutputTracker: MapOutputTrackerMaster, blockManagerMaster: BlockManagerMaster, @@ -65,6 +68,7 @@ class DAGScheduler( def this(sc: SparkContext, taskScheduler: TaskScheduler) = { this( + sc, taskScheduler, sc.listenerBus, sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], @@ -74,8 +78,6 @@ class DAGScheduler( def this(sc: SparkContext) = this(sc, sc.taskScheduler) - private var eventProcessActor: ActorRef = _ - private[scheduler] val nextJobId = new AtomicInteger(0) private[scheduler] def numTotalJobs: Int = nextJobId.get() private val nextStageId = new AtomicInteger(0) @@ -113,50 +115,31 @@ class DAGScheduler( // stray messages to detect. private val failedEpoch = new HashMap[String, Long] - taskScheduler.setDAGScheduler(this) + private val dagSchedulerActorSupervisor = + env.actorSystem.actorOf(Props(new DAGSchedulerActorSupervisor(this))) - /** - * Starts the event processing actor. The actor has two responsibilities: - * - * 1. Waits for events like job submission, task finished, task failure etc., and calls - * [[org.apache.spark.scheduler.DAGScheduler.processEvent()]] to process them. - * 2. Schedules a periodical task to resubmit failed stages. - * - * NOTE: the actor cannot be started in the constructor, because the periodical task references - * some internal states of the enclosing [[org.apache.spark.scheduler.DAGScheduler]] object, thus - * cannot be scheduled until the [[org.apache.spark.scheduler.DAGScheduler]] is fully constructed. - */ - def start() { - eventProcessActor = env.actorSystem.actorOf(Props(new Actor { - /** - * The main event loop of the DAG scheduler. - */ - def receive = { - case event: DAGSchedulerEvent => - logTrace("Got event of type " + event.getClass.getName) - - /** - * All events are forwarded to `processEvent()`, so that the event processing logic can - * easily tested without starting a dedicated actor. Please refer to `DAGSchedulerSuite` - * for details. - */ - if (!processEvent(event)) { - submitWaitingStages() - } else { - context.stop(self) - } - } - })) + private[scheduler] var eventProcessActor: ActorRef = _ + + private def initializeEventProcessActor() { + // blocking the thread until supervisor is started, which ensures eventProcessActor is + // not null before any job is submitted + implicit val timeout = Timeout(30 seconds) + val initEventActorReply = + dagSchedulerActorSupervisor ? Props(new DAGSchedulerEventProcessActor(this)) + eventProcessActor = Await.result(initEventActorReply, timeout.duration). + asInstanceOf[ActorRef] } + initializeEventProcessActor() + // Called by TaskScheduler to report task's starting. def taskStarted(task: Task[_], taskInfo: TaskInfo) { eventProcessActor ! BeginEvent(task, taskInfo) } // Called to report that a task has completed and results are being fetched remotely. - def taskGettingResult(task: Task[_], taskInfo: TaskInfo) { - eventProcessActor ! GettingResultEvent(task, taskInfo) + def taskGettingResult(taskInfo: TaskInfo) { + eventProcessActor ! GettingResultEvent(taskInfo) } // Called by TaskScheduler to report task completions or failures. @@ -436,7 +419,7 @@ class DAGScheduler( { // Check to make sure we are not launching a task on a partition that does not exist. val maxPartitions = rdd.partitions.length - partitions.find(p => p >= maxPartitions).foreach { p => + partitions.find(p => p >= maxPartitions || p < 0).foreach { p => throw new IllegalArgumentException( "Attempting to access a non-existent partition: " + p + ". " + "Total number of partitions: " + maxPartitions) @@ -511,6 +494,15 @@ class DAGScheduler( eventProcessActor ! AllJobsCancelled } + private[scheduler] def doCancelAllJobs() { + // Cancel all running jobs. + runningStages.map(_.jobId).foreach(handleJobCancellation(_, + reason = "as part of cancellation of all jobs")) + activeJobs.clear() // These should already be empty by this point, + jobIdToActiveJob.clear() // but just in case we lost track of some jobs... + submitWaitingStages() + } + /** * Cancel all jobs associated with a running or scheduled stage. */ @@ -518,148 +510,30 @@ class DAGScheduler( eventProcessActor ! StageCancelled(stageId) } - /** - * Process one event retrieved from the event processing actor. - * - * @param event The event to be processed. - * @return `true` if we should stop the event loop. - */ - private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = { - event match { - case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) => - var finalStage: Stage = null - try { - // New stage creation may throw an exception if, for example, jobs are run on a HadoopRDD - // whose underlying HDFS files have been deleted. - finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite)) - } catch { - case e: Exception => - logWarning("Creating new stage failed due to exception - job: " + jobId, e) - listener.jobFailed(e) - return false - } - val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties) - clearCacheLocs() - logInfo("Got job " + job.jobId + " (" + callSite + ") with " + partitions.length + - " output partitions (allowLocal=" + allowLocal + ")") - logInfo("Final stage: " + finalStage + " (" + finalStage.name + ")") - logInfo("Parents of final stage: " + finalStage.parents) - logInfo("Missing parents: " + getMissingParentStages(finalStage)) - if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { - // Compute very short actions like first() or take() with no parent stages locally. - listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) - runLocally(job) - } else { - jobIdToActiveJob(jobId) = job - activeJobs += job - resultStageToJob(finalStage) = job - listenerBus.post( - SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, properties)) - submitStage(finalStage) - } - - case StageCancelled(stageId) => - handleStageCancellation(stageId) - - case JobCancelled(jobId) => - handleJobCancellation(jobId) - - case JobGroupCancelled(groupId) => - // Cancel all jobs belonging to this job group. - // First finds all active jobs with this group id, and then kill stages for them. - val activeInGroup = activeJobs.filter(activeJob => - groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) - val jobIds = activeInGroup.map(_.jobId) - jobIds.foreach(jobId => handleJobCancellation(jobId, - "as part of cancelled job group %s".format(groupId))) - - case AllJobsCancelled => - // Cancel all running jobs. - runningStages.map(_.jobId).foreach(jobId => handleJobCancellation(jobId, - "as part of cancellation of all jobs")) - activeJobs.clear() // These should already be empty by this point, - jobIdToActiveJob.clear() // but just in case we lost track of some jobs... - - case ExecutorAdded(execId, host) => - handleExecutorAdded(execId, host) - - case ExecutorLost(execId) => - handleExecutorLost(execId) - - case BeginEvent(task, taskInfo) => - for ( - stage <- stageIdToStage.get(task.stageId); - stageInfo <- stageToInfos.get(stage) - ) { - if (taskInfo.serializedSize > TASK_SIZE_TO_WARN * 1024 && - !stageInfo.emittedTaskSizeWarning) { - stageInfo.emittedTaskSizeWarning = true - logWarning(("Stage %d (%s) contains a task of very large " + - "size (%d KB). The maximum recommended task size is %d KB.").format( - task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, TASK_SIZE_TO_WARN)) - } - } - listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) - - case GettingResultEvent(task, taskInfo) => - listenerBus.post(SparkListenerTaskGettingResult(taskInfo)) - - case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) => - val stageId = task.stageId - val taskType = Utils.getFormattedClassName(task) - listenerBus.post(SparkListenerTaskEnd(stageId, taskType, reason, taskInfo, taskMetrics)) - handleTaskCompletion(completion) - - case TaskSetFailed(taskSet, reason) => - stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) } - - case ResubmitFailedStages => - if (failedStages.size > 0) { - // Failed stages may be removed by job cancellation, so failed might be empty even if - // the ResubmitFailedStages event has been scheduled. - resubmitFailedStages() - } - - case StopDAGScheduler => - // Cancel any active jobs - for (job <- activeJobs) { - val error = new SparkException("Job cancelled because SparkContext was shut down") - job.listener.jobFailed(error) - // Tell the listeners that all of the running stages have ended. Don't bother - // cancelling the stages because if the DAG scheduler is stopped, the entire application - // is in the process of getting stopped. - val stageFailedMessage = "Stage cancelled because SparkContext was shut down" - runningStages.foreach { stage => - val info = stageToInfos(stage) - info.stageFailed(stageFailedMessage) - listenerBus.post(SparkListenerStageCompleted(info)) - } - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) - } - return true - } - false - } - /** * Resubmit any failed stages. Ordinarily called after a small amount of time has passed since * the last fetch failure. */ private[scheduler] def resubmitFailedStages() { - logInfo("Resubmitting failed stages") - clearCacheLocs() - val failedStagesCopy = failedStages.toArray - failedStages.clear() - for (stage <- failedStagesCopy.sortBy(_.jobId)) { - submitStage(stage) + if (failedStages.size > 0) { + // Failed stages may be removed by job cancellation, so failed might be empty even if + // the ResubmitFailedStages event has been scheduled. + logInfo("Resubmitting failed stages") + clearCacheLocs() + val failedStagesCopy = failedStages.toArray + failedStages.clear() + for (stage <- failedStagesCopy.sortBy(_.jobId)) { + submitStage(stage) + } } + submitWaitingStages() } /** * Check for waiting or failed stages which are now eligible for resubmission. * Ordinarily run on every iteration of the event loop. */ - private[scheduler] def submitWaitingStages() { + private def submitWaitingStages() { // TODO: We might want to run this less often, when we are sure that something has become // runnable that wasn't before. logTrace("Checking for newly runnable parent stages") @@ -730,6 +604,102 @@ class DAGScheduler( } } + private[scheduler] def handleJobGroupCancelled(groupId: String) { + // Cancel all jobs belonging to this job group. + // First finds all active jobs with this group id, and then kill stages for them. + val activeInGroup = activeJobs.filter(activeJob => + groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) + val jobIds = activeInGroup.map(_.jobId) + jobIds.foreach(handleJobCancellation(_, "part of cancelled job group %s".format(groupId))) + submitWaitingStages() + } + + private[scheduler] def handleBeginEvent(task: Task[_], taskInfo: TaskInfo) { + for (stage <- stageIdToStage.get(task.stageId); stageInfo <- stageToInfos.get(stage)) { + if (taskInfo.serializedSize > DAGScheduler.TASK_SIZE_TO_WARN * 1024 && + !stageInfo.emittedTaskSizeWarning) { + stageInfo.emittedTaskSizeWarning = true + logWarning(("Stage %d (%s) contains a task of very large " + + "size (%d KB). The maximum recommended task size is %d KB.").format( + task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, + DAGScheduler.TASK_SIZE_TO_WARN)) + } + } + listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) + submitWaitingStages() + } + + private[scheduler] def handleTaskSetFailed(taskSet: TaskSet, reason: String) { + stageIdToStage.get(taskSet.stageId).foreach {abortStage(_, reason) } + submitWaitingStages() + } + + private[scheduler] def cleanUpAfterSchedulerStop() { + for (job <- activeJobs) { + val error = new SparkException("Job cancelled because SparkContext was shut down") + job.listener.jobFailed(error) + // Tell the listeners that all of the running stages have ended. Don't bother + // cancelling the stages because if the DAG scheduler is stopped, the entire application + // is in the process of getting stopped. + val stageFailedMessage = "Stage cancelled because SparkContext was shut down" + runningStages.foreach { stage => + val info = stageToInfos(stage) + info.stageFailed(stageFailedMessage) + listenerBus.post(SparkListenerStageCompleted(info)) + } + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) + } + } + + private[scheduler] def handleGetTaskResult(taskInfo: TaskInfo) { + listenerBus.post(SparkListenerTaskGettingResult(taskInfo)) + submitWaitingStages() + } + + private[scheduler] def handleJobSubmitted(jobId: Int, + finalRDD: RDD[_], + func: (TaskContext, Iterator[_]) => _, + partitions: Array[Int], + allowLocal: Boolean, + callSite: String, + listener: JobListener, + properties: Properties = null) + { + var finalStage: Stage = null + try { + // New stage creation may throw an exception if, for example, jobs are run on a + // HadoopRDD whose underlying HDFS files have been deleted. + finalStage = newStage(finalRDD, partitions.size, None, jobId, Some(callSite)) + } catch { + case e: Exception => + logWarning("Creating new stage failed due to exception - job: " + jobId, e) + listener.jobFailed(e) + return + } + if (finalStage != null) { + val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties) + clearCacheLocs() + logInfo("Got job %s (%s) with %d output partitions (allowLocal=%s)".format( + job.jobId, callSite, partitions.length, allowLocal)) + logInfo("Final stage: " + finalStage + "(" + finalStage.name + ")") + logInfo("Parents of final stage: " + finalStage.parents) + logInfo("Missing parents: " + getMissingParentStages(finalStage)) + if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { + // Compute very short actions like first() or take() with no parent stages locally. + listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) + runLocally(job) + } else { + jobIdToActiveJob(jobId) = job + activeJobs += job + resultStageToJob(finalStage) = job + listenerBus.post(SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, + properties)) + submitStage(finalStage) + } + } + submitWaitingStages() + } + /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { val jobId = activeJobForStage(stage) @@ -819,9 +789,12 @@ class DAGScheduler( * Responds to a task finishing. This is called inside the event loop so it assumes that it can * modify the scheduler's internal state. Use taskEnded() to post a task end event from outside. */ - private def handleTaskCompletion(event: CompletionEvent) { + private[scheduler] def handleTaskCompletion(event: CompletionEvent) { val task = event.task - + val stageId = task.stageId + val taskType = Utils.getFormattedClassName(task) + listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, event.taskInfo, + event.taskMetrics)) if (!stageIdToStage.contains(task.stageId)) { // Skip all the actions if the stage has been cancelled. return @@ -964,6 +937,7 @@ class DAGScheduler( // Unrecognized failure - also do nothing. If the task fails repeatedly, the TaskScheduler // will abort the job. } + submitWaitingStages() } /** @@ -973,7 +947,7 @@ class DAGScheduler( * Optionally the epoch during which the failure was caught can be passed to avoid allowing * stray fetch failures from possibly retriggering the detection of a node as lost. */ - private def handleExecutorLost(execId: String, maybeEpoch: Option[Long] = None) { + private[scheduler] def handleExecutorLost(execId: String, maybeEpoch: Option[Long] = None) { val currentEpoch = maybeEpoch.getOrElse(mapOutputTracker.getEpoch) if (!failedEpoch.contains(execId) || failedEpoch(execId) < currentEpoch) { failedEpoch(execId) = currentEpoch @@ -993,17 +967,19 @@ class DAGScheduler( logDebug("Additional executor lost message for " + execId + "(epoch " + currentEpoch + ")") } + submitWaitingStages() } - private def handleExecutorAdded(execId: String, host: String) { + private[scheduler] def handleExecutorAdded(execId: String, host: String) { // remove from failedEpoch(execId) ? if (failedEpoch.contains(execId)) { logInfo("Host added was in lost list earlier: " + host) failedEpoch -= execId } + submitWaitingStages() } - private def handleStageCancellation(stageId: Int) { + private[scheduler] def handleStageCancellation(stageId: Int) { if (stageIdToJobIds.contains(stageId)) { val jobsThatUseStage: Array[Int] = stageIdToJobIds(stageId).toArray jobsThatUseStage.foreach(jobId => { @@ -1012,22 +988,24 @@ class DAGScheduler( } else { logInfo("No active jobs to kill for Stage " + stageId) } + submitWaitingStages() } - private def handleJobCancellation(jobId: Int, reason: String = "") { + private[scheduler] def handleJobCancellation(jobId: Int, reason: String = "") { if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { failJobAndIndependentStages(jobIdToActiveJob(jobId), "Job %d cancelled %s".format(jobId, reason), None) } + submitWaitingStages() } /** * Aborts all jobs depending on a particular Stage. This is called in response to a task set * being canceled by the TaskScheduler. Use taskSetFailed() to inject this event from outside. */ - private def abortStage(failedStage: Stage, reason: String) { + private[scheduler] def abortStage(failedStage: Stage, reason: String) { if (!stageIdToStage.contains(failedStage.id)) { // Skip all the actions if the stage has been removed. return @@ -1156,13 +1134,88 @@ class DAGScheduler( } def stop() { - if (eventProcessActor != null) { - eventProcessActor ! StopDAGScheduler - } + logInfo("Stopping DAGScheduler") + dagSchedulerActorSupervisor ! PoisonPill taskScheduler.stop() } } +private[scheduler] class DAGSchedulerActorSupervisor(dagScheduler: DAGScheduler) + extends Actor with Logging { + + override val supervisorStrategy = + OneForOneStrategy() { + case x: Exception => + logError("eventProcesserActor failed due to the error %s; shutting down SparkContext" + .format(x.getMessage)) + dagScheduler.doCancelAllJobs() + dagScheduler.sc.stop() + Stop + } + + def receive = { + case p: Props => sender ! context.actorOf(p) + case _ => logWarning("received unknown message in DAGSchedulerActorSupervisor") + } +} + +private[scheduler] class DAGSchedulerEventProcessActor(dagScheduler: DAGScheduler) + extends Actor with Logging { + + override def preStart() { + // set DAGScheduler for taskScheduler to ensure eventProcessActor is always + // valid when the messages arrive + dagScheduler.taskScheduler.setDAGScheduler(dagScheduler) + } + + /** + * The main event loop of the DAG scheduler. + */ + def receive = { + case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) => + dagScheduler.handleJobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, + listener, properties) + + case StageCancelled(stageId) => + dagScheduler.handleStageCancellation(stageId) + + case JobCancelled(jobId) => + dagScheduler.handleJobCancellation(jobId) + + case JobGroupCancelled(groupId) => + dagScheduler.handleJobGroupCancelled(groupId) + + case AllJobsCancelled => + dagScheduler.doCancelAllJobs() + + case ExecutorAdded(execId, host) => + dagScheduler.handleExecutorAdded(execId, host) + + case ExecutorLost(execId) => + dagScheduler.handleExecutorLost(execId) + + case BeginEvent(task, taskInfo) => + dagScheduler.handleBeginEvent(task, taskInfo) + + case GettingResultEvent(taskInfo) => + dagScheduler.handleGetTaskResult(taskInfo) + + case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) => + dagScheduler.handleTaskCompletion(completion) + + case TaskSetFailed(taskSet, reason) => + dagScheduler.handleTaskSetFailed(taskSet, reason) + + case ResubmitFailedStages => + dagScheduler.resubmitFailedStages() + } + + override def postStop() { + // Cancel any active jobs in postStop hook + dagScheduler.cleanUpAfterSchedulerStop() + } +} + private[spark] object DAGScheduler { // The time, in millis, to wait for fetch failure events to stop coming in after one is detected; // this is a simplistic way to avoid resubmitting tasks in the non-fetchable map stage one by one diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 0800c5684c60f..23f57441b4b11 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -57,7 +57,7 @@ private[scheduler] case class BeginEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent private[scheduler] -case class GettingResultEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent +case class GettingResultEvent(taskInfo: TaskInfo) extends DAGSchedulerEvent private[scheduler] case class CompletionEvent( task: Task[_], @@ -76,5 +76,3 @@ private[scheduler] case class TaskSetFailed(taskSet: TaskSet, reason: String) extends DAGSchedulerEvent private[scheduler] case object ResubmitFailedStages extends DAGSchedulerEvent - -private[scheduler] case object StopDAGScheduler extends DAGSchedulerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index a81b834267f0d..f3bd0797aa035 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -465,7 +465,7 @@ private[spark] class TaskSetManager( def handleTaskGettingResult(tid: Long) = { val info = taskInfos(tid) info.markGettingResult() - sched.dagScheduler.taskGettingResult(tasks(info.index), info) + sched.dagScheduler.taskGettingResult(info) } /** diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index ff69eb7e53f8e..d172dd1ac8e1b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -21,6 +21,8 @@ import scala.Tuple2 import scala.collection.mutable.{HashSet, HashMap, Map} import scala.language.reflectiveCalls +import akka.actor._ +import akka.testkit.{ImplicitSender, TestKit, TestActorRef} import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark._ @@ -28,19 +30,16 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} -/** - * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler - * rather than spawning an event loop thread as happens in the real code. They use EasyMock - * to mock out two classes that DAGScheduler interacts with: TaskScheduler (to which TaskSets are - * submitted) and BlockManagerMaster (from which cache locations are retrieved and to which dead - * host notifications are sent). In addition, tests may check for side effects on a non-mocked - * MapOutputTracker instance. - * - * Tests primarily consist of running DAGScheduler#processEvent and - * DAGScheduler#submitWaitingStages (via test utility functions like runEvent or respondToTaskSet) - * and capturing the resulting TaskSets from the mock TaskScheduler. - */ -class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { +class BuggyDAGEventProcessActor extends Actor { + val state = 0 + def receive = { + case _ => throw new SparkException("error") + } +} + +class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with FunSuite + with ImplicitSender with BeforeAndAfter with LocalSparkContext { + val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() @@ -82,6 +81,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont var mapOutputTracker: MapOutputTrackerMaster = null var scheduler: DAGScheduler = null + var dagEventProcessTestActor: TestActorRef[DAGSchedulerEventProcessActor] = null /** * Set of cache locations to return from our mock BlockManagerMaster. @@ -121,6 +121,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont results.clear() mapOutputTracker = new MapOutputTrackerMaster(conf) scheduler = new DAGScheduler( + sc, taskScheduler, sc.listenerBus, mapOutputTracker, @@ -131,10 +132,13 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont runLocallyWithinThread(job) } } + dagEventProcessTestActor = TestActorRef[DAGSchedulerEventProcessActor]( + Props(classOf[DAGSchedulerEventProcessActor], scheduler))(system) } - after { - scheduler.stop() + override def afterAll() { + super.afterAll() + TestKit.shutdownActorSystem(system) } /** @@ -178,8 +182,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont * DAGScheduler event loop. */ private def runEvent(event: DAGSchedulerEvent) { - assert(!scheduler.processEvent(event)) - scheduler.submitWaitingStages() + dagEventProcessTestActor.receive(event) } /** @@ -209,7 +212,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont listener: JobListener = jobListener): Int = { val jobId = scheduler.nextJobId.getAndIncrement() runEvent(JobSubmitted(jobId, rdd, func, partitions, allowLocal, null, listener)) - return jobId + jobId } /** Sends TaskSetFailed to the scheduler. */ @@ -223,19 +226,17 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont } test("zero split job") { - val rdd = makeRdd(0, Nil) var numResults = 0 val fakeListener = new JobListener() { override def taskSucceeded(partition: Int, value: Any) = numResults += 1 override def jobFailed(exception: Exception) = throw exception } - submit(rdd, Array(), listener = fakeListener) + submit(makeRdd(0, Nil), Array(), listener = fakeListener) assert(numResults === 0) } test("run trivial job") { - val rdd = makeRdd(1, Nil) - submit(rdd, Array(0)) + submit(makeRdd(1, Nil), Array(0)) complete(taskSets(0), List((Success, 42))) assert(results === Map(0 -> 42)) assertDataStructuresEmpty @@ -529,6 +530,18 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assertDataStructuresEmpty } + test("DAGSchedulerActorSupervisor closes the SparkContext when EventProcessActor crashes") { + val actorSystem = ActorSystem("test") + val supervisor = actorSystem.actorOf( + Props(classOf[DAGSchedulerActorSupervisor], scheduler), "dagSupervisor") + supervisor ! Props[BuggyDAGEventProcessActor] + val child = expectMsgType[ActorRef] + watch(child) + child ! "hi" + expectMsgPF(){ case Terminated(child) => () } + assert(scheduler.sc.dagScheduler === null) + } + /** * Assert that the supplied TaskSet has exactly the given hosts as its preferred locations. * Note that this checks only the host and not the executor ID. @@ -561,3 +574,4 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assert(scheduler.waitingStages.isEmpty) } } + diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 2fb750d9ee378..a8b605c5b212a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -305,7 +305,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} override def executorAdded(execId: String, host: String) {} } - + taskScheduler.setDAGScheduler(dagScheduler) // Give zero core offers. Should not generate any tasks val zeroCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", 0), new WorkerOffer("executor1", "host1", 0)) From 87cf35c2d6acc9649b3fb05648b79b9862b3959b Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 25 Apr 2014 17:55:04 -0700 Subject: [PATCH 184/641] SPARK-1632. Remove unnecessary boxing in compares in ExternalAppendOnlyM... ...ap Author: Sandy Ryza Closes #559 from sryza/sandy-spark-1632 and squashes the following commits: a6cd352 [Sandy Ryza] Only compute hashes once 04e3884 [Sandy Ryza] SPARK-1632. Remove unnecessary boxing in compares in ExternalAppendOnlyMap --- .../spark/util/collection/ExternalAppendOnlyMap.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) 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 d615767284c0b..170f09be21534 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 @@ -337,8 +337,8 @@ class ExternalAppendOnlyMap[K, V, C]( } override def compareTo(other: StreamBuffer): Int = { - // minus sign because mutable.PriorityQueue dequeues the max, not the min - -minKeyHash.compareTo(other.minKeyHash) + // descending order because mutable.PriorityQueue dequeues the max, not the min + if (other.minKeyHash < minKeyHash) -1 else if (other.minKeyHash == minKeyHash) 0 else 1 } } } @@ -422,7 +422,9 @@ class ExternalAppendOnlyMap[K, V, C]( private[spark] object ExternalAppendOnlyMap { private class KCComparator[K, C] extends Comparator[(K, C)] { def compare(kc1: (K, C), kc2: (K, C)): Int = { - kc1._1.hashCode().compareTo(kc2._1.hashCode()) + val hash1 = kc1._1.hashCode() + val hash2 = kc2._1.hashCode() + if (hash1 < hash2) -1 else if (hash1 == hash2) 0 else 1 } } } From 058797c1722c9251f6bc6ad2672cb0e79146b04f Mon Sep 17 00:00:00 2001 From: zsxwing Date: Fri, 25 Apr 2014 19:04:34 -0700 Subject: [PATCH 185/641] [Spark-1382] Fix NPE in DStream.slice (updated version of #365) @zsxwing I cherry-picked your changes and merged the master. #365 had some conflicts once again! Author: zsxwing Author: Tathagata Das Closes #562 from tdas/SPARK-1382 and squashes the following commits: e2962c1 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-1382 20968d9 [zsxwing] Replace Exception with SparkException in DStream e476651 [zsxwing] Merge remote-tracking branch 'origin/master' into SPARK-1382 35ba56a [zsxwing] SPARK-1382: Fix NPE in DStream.slice --- .../spark/streaming/dstream/DStream.scala | 22 ++++++++++--------- .../streaming/BasicOperationsSuite.scala | 12 +++++++++- 2 files changed, 23 insertions(+), 11 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index f69f69e0c44af..4709a62381647 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -18,20 +18,19 @@ package org.apache.spark.streaming.dstream +import java.io.{IOException, ObjectInputStream, ObjectOutputStream} + import scala.deprecated import scala.collection.mutable.HashMap import scala.reflect.ClassTag -import java.io.{IOException, ObjectInputStream, ObjectOutputStream} - -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.MetadataCleaner import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.scheduler.Job -import org.apache.spark.streaming.Duration +import org.apache.spark.util.MetadataCleaner /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous @@ -144,7 +143,7 @@ abstract class DStream[T: ClassTag] ( */ private[streaming] def initialize(time: Time) { if (zeroTime != null && zeroTime != time) { - throw new Exception("ZeroTime is already initialized to " + zeroTime + throw new SparkException("ZeroTime is already initialized to " + zeroTime + ", cannot initialize it again to " + time) } zeroTime = time @@ -220,7 +219,7 @@ abstract class DStream[T: ClassTag] ( "which requires " + this.getClass.getSimpleName + " to remember generated RDDs for more " + "than " + rememberDuration.milliseconds / 1000 + " seconds. But Spark's metadata cleanup" + "delay is set to " + metadataCleanerDelay + " seconds, which is not sufficient. Please " + - "set the Java property 'spark.cleaner.delay' to more than " + + "set the Java cleaner delay to more than " + math.ceil(rememberDuration.milliseconds / 1000.0).toInt + " seconds." ) @@ -235,7 +234,7 @@ abstract class DStream[T: ClassTag] ( private[streaming] def setContext(s: StreamingContext) { if (ssc != null && ssc != s) { - throw new Exception("Context is already set in " + this + ", cannot set it again") + throw new SparkException("Context is already set in " + this + ", cannot set it again") } ssc = s logInfo("Set context for " + this) @@ -244,7 +243,7 @@ abstract class DStream[T: ClassTag] ( private[streaming] def setGraph(g: DStreamGraph) { if (graph != null && graph != g) { - throw new Exception("Graph is already set in " + this + ", cannot set it again") + throw new SparkException("Graph is already set in " + this + ", cannot set it again") } graph = g dependencies.foreach(_.setGraph(graph)) @@ -261,7 +260,7 @@ abstract class DStream[T: ClassTag] ( /** Checks whether the 'time' is valid wrt slideDuration for generating RDD */ private[streaming] def isTimeValid(time: Time): Boolean = { if (!isInitialized) { - throw new Exception (this + " has not been initialized") + throw new SparkException (this + " has not been initialized") } else if (time <= zeroTime || ! (time - zeroTime).isMultipleOf(slideDuration)) { logInfo("Time " + time + " is invalid as zeroTime is " + zeroTime + " and slideDuration is " + slideDuration + " and difference is " + (time - zeroTime)) @@ -728,6 +727,9 @@ abstract class DStream[T: ClassTag] ( * Return all the RDDs between 'fromTime' to 'toTime' (both included) */ def slice(fromTime: Time, toTime: Time): Seq[RDD[T]] = { + if (!isInitialized) { + throw new SparkException(this + " has not been initialized") + } if (!(fromTime - zeroTime).isMultipleOf(slideDuration)) { logWarning("fromTime (" + fromTime + ") is not a multiple of slideDuration (" + slideDuration + ")") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 4792ca1f8ae3e..04925886c39e4 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.SparkContext._ import util.ManualClock -import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.{SparkException, SparkConf} import org.apache.spark.streaming.dstream.{WindowedDStream, DStream} import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} import scala.reflect.ClassTag @@ -398,6 +398,16 @@ class BasicOperationsSuite extends TestSuiteBase { Thread.sleep(1000) } + test("slice - has not been initialized") { + val ssc = new StreamingContext(conf, Seconds(1)) + val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4)) + val stream = new TestInputStream[Int](ssc, input, 2) + val thrown = intercept[SparkException] { + stream.slice(new Time(0), new Time(1000)) + } + assert(thrown.getMessage.contains("has not been initialized")) + } + val cleanupTestInput = (0 until 10).map(x => Seq(x, x + 1)).toSeq test("rdd cleanup - map and window") { From e6e44e46e345640fd89be9f10cdc448b44d30cc1 Mon Sep 17 00:00:00 2001 From: wangfei Date: Sat, 26 Apr 2014 14:38:42 -0700 Subject: [PATCH 186/641] add note of how to support table with more than 22 fields Author: wangfei Closes #564 from scwf/patch-6 and squashes the following commits: a331876 [wangfei] Update sql-programming-guide.md 685135b [wangfei] Update sql-programming-guide.md 10b3dc0 [wangfei] Update sql-programming-guide.md 1c40480 [wangfei] add note of how to support table with 22 fields --- docs/sql-programming-guide.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 3ca6cc95bd2db..0c743c9d6010d 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -114,6 +114,8 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext._ // Define the schema using a case class. +// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, +// you can use custom classes that implement the Product interface. case class Person(name: String, age: Int) // Create an RDD of Person objects and register it as a table. From 8e37ed6eb81687140b6cdb00f4ec609ec7ba9be1 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Sat, 26 Apr 2014 14:39:54 -0700 Subject: [PATCH 187/641] [SPARK-1608] [SQL] Fix Cast.nullable when cast from StringType to NumericType/TimestampType. `Cast.nullable` should be `true` when cast from `StringType` to `NumericType` or `TimestampType`. Because if `StringType` expression has an illegal number string or illegal timestamp string, the casted value becomes `null`. Author: Takuya UESHIN Closes #532 from ueshin/issues/SPARK-1608 and squashes the following commits: 065d37c [Takuya UESHIN] Add tests to check nullabilities of cast expressions. f278ed7 [Takuya UESHIN] Revert test to keep it readable and concise. 9fc9380 [Takuya UESHIN] Fix Cast.nullable when cast from StringType to NumericType/TimestampType. --- .../apache/spark/sql/catalyst/expressions/Cast.scala | 6 +++++- .../expressions/ExpressionEvaluationSuite.scala | 12 ++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 8b79b0cd65a84..40d2b42a0cda3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -24,7 +24,11 @@ import org.apache.spark.sql.catalyst.types._ /** Cast the child expression to the target data type. */ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { override def foldable = child.foldable - def nullable = child.nullable + def nullable = (child.dataType, dataType) match { + case (StringType, _: NumericType) => true + case (StringType, TimestampType) => true + case _ => child.nullable + } override def toString = s"CAST($child, $dataType)" type EvaluatedType = Any diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 4ce0dff9e1586..d287ad73b9e9f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -245,6 +245,18 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Literal(23.toShort) + Cast(true, ShortType), 24) intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)} + + assert(("abcdef" cast StringType).nullable === false) + assert(("abcdef" cast BinaryType).nullable === false) + assert(("abcdef" cast BooleanType).nullable === false) + assert(("abcdef" cast TimestampType).nullable === true) + assert(("abcdef" cast LongType).nullable === true) + assert(("abcdef" cast IntegerType).nullable === true) + assert(("abcdef" cast ShortType).nullable === true) + assert(("abcdef" cast ByteType).nullable === true) + assert(("abcdef" cast DecimalType).nullable === true) + assert(("abcdef" cast DoubleType).nullable === true) + assert(("abcdef" cast FloatType).nullable === true) } test("timestamp") { From 762af4e9c2837c8ca125838b2ca01e328ee6669f Mon Sep 17 00:00:00 2001 From: Sandeep Date: Sat, 26 Apr 2014 19:04:33 -0700 Subject: [PATCH 188/641] SPARK-1467: Make StorageLevel.apply() factory methods Developer APIs We may want to evolve these in the future to add things like SSDs, so let's mark them as experimental for now. Long-term the right solution might be some kind of builder. The stable API should be the existing StorageLevel constants. Author: Sandeep Closes #551 from techaddict/SPARK-1467 and squashes the following commits: 6bdda24 [Sandeep] SPARK-1467: Make StorageLevel.apply() factory methods as Developer Api's We may want to evolve these in the future to add things like SSDs, so let's mark them as experimental for now. Long-term the right solution might be some kind of builder. The stable API should be the existing StorageLevel constants. --- .../apache/spark/storage/StorageLevel.scala | 26 ++++++++++++++++--- 1 file changed, 22 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 95e71de2d3f1d..c9a52e0366d93 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -19,6 +19,8 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} +import org.apache.spark.annotation.DeveloperApi + /** * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, * or Tachyon, whether to drop the RDD to disk if it falls out of memory or Tachyon , whether to @@ -142,21 +144,37 @@ object StorageLevel { val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) val OFF_HEAP = new StorageLevel(false, false, true, false) - /** Create a new StorageLevel object without setting useOffHeap */ + /** + * :: DeveloperApi :: + * Create a new StorageLevel object without setting useOffHeap + */ + @DeveloperApi def apply(useDisk: Boolean, useMemory: Boolean, useOffHeap: Boolean, deserialized: Boolean, replication: Int) = getCachedStorageLevel( new StorageLevel(useDisk, useMemory, useOffHeap, deserialized, replication)) - /** Create a new StorageLevel object */ + /** + * :: DeveloperApi :: + * Create a new StorageLevel object + */ + @DeveloperApi def apply(useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, replication: Int = 1) = getCachedStorageLevel( new StorageLevel(useDisk, useMemory, false, deserialized, replication)) - /** Create a new StorageLevel object from its integer representation */ + /** + * :: DeveloperApi :: + * Create a new StorageLevel object from its integer representation + */ + @DeveloperApi def apply(flags: Int, replication: Int): StorageLevel = getCachedStorageLevel(new StorageLevel(flags, replication)) - /** Read StorageLevel object from ObjectInput stream */ + /** + * :: DeveloperApi :: + * Read StorageLevel object from ObjectInput stream + */ + @DeveloperApi def apply(in: ObjectInput): StorageLevel = { val obj = new StorageLevel() obj.readExternal(in) From aa9a7f5db7bffcdbcd7fd53694c606d3a8cdd21f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 26 Apr 2014 19:24:29 -0700 Subject: [PATCH 189/641] SPARK-1606: Infer user application arguments instead of requiring --arg. This modifies spark-submit to do something more like the Hadoop `jar` command. Now we have the following syntax: ./bin/spark-submit [options] user.jar [user options] Author: Patrick Wendell Closes #563 from pwendell/spark-submit and squashes the following commits: 32241fc [Patrick Wendell] Review feedback 3adfb69 [Patrick Wendell] Small fix bc48139 [Patrick Wendell] SPARK-1606: Infer user application arguments instead of requiring --arg. --- .../org/apache/spark/deploy/SparkSubmit.scala | 1 - .../spark/deploy/SparkSubmitArguments.scala | 226 +++++++++--------- .../scala/org/apache/spark/util/Utils.scala | 7 + .../spark/deploy/SparkSubmitSuite.scala | 77 +++--- docs/cluster-overview.md | 22 +- docs/quick-start.md | 10 +- 6 files changed, 181 insertions(+), 162 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 58aa6d951a204..24edc60684376 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -185,7 +185,6 @@ object SparkSubmit { if (clusterManager == STANDALONE) { val existingJars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq()) sysProps.put("spark.jars", (existingJars ++ Seq(appArgs.primaryResource)).mkString(",")) - println("SPARK JARS" + sysProps.get("spark.jars")) } if (deployOnCluster && clusterManager == STANDALONE) { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index c545b093ac82e..58d9e9add764a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -21,14 +21,15 @@ import java.io.{File, FileInputStream, IOException} import java.util.Properties import scala.collection.JavaConversions._ -import scala.collection.mutable.{HashMap, ArrayBuffer} +import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.spark.SparkException +import org.apache.spark.util.Utils /** * Parses and encapsulates arguments from the spark-submit script. */ -private[spark] class SparkSubmitArguments(args: Array[String]) { +private[spark] class SparkSubmitArguments(args: Seq[String]) { var master: String = null var deployMode: String = null var executorMemory: String = null @@ -118,8 +119,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { if (master.startsWith("yarn")) { val hasHadoopEnv = sys.env.contains("HADOOP_CONF_DIR") || sys.env.contains("YARN_CONF_DIR") - val testing = sys.env.contains("SPARK_TESTING") - if (!hasHadoopEnv && !testing) { + if (!hasHadoopEnv && !Utils.isTesting) { throw new Exception(s"When running with master '$master' " + "either HADOOP_CONF_DIR or YARN_CONF_DIR must be set in the environment.") } @@ -156,119 +156,121 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { """.stripMargin } - private def parseOpts(opts: List[String]): Unit = opts match { - case ("--name") :: value :: tail => - name = value - parseOpts(tail) + /** Fill in values by parsing user options. */ + private def parseOpts(opts: Seq[String]): Unit = { + // Delineates parsing of Spark options from parsing of user options. + var inSparkOpts = true + parse(opts) - case ("--master") :: value :: tail => - master = value - parseOpts(tail) + def parse(opts: Seq[String]): Unit = opts match { + case ("--name") :: value :: tail => + name = value + parse(tail) - case ("--class") :: value :: tail => - mainClass = value - parseOpts(tail) + case ("--master") :: value :: tail => + master = value + parse(tail) - case ("--deploy-mode") :: value :: tail => - if (value != "client" && value != "cluster") { - SparkSubmit.printErrorAndExit("--deploy-mode must be either \"client\" or \"cluster\"") - } - deployMode = value - parseOpts(tail) - - case ("--num-executors") :: value :: tail => - numExecutors = value - parseOpts(tail) - - case ("--total-executor-cores") :: value :: tail => - totalExecutorCores = value - parseOpts(tail) - - case ("--executor-cores") :: value :: tail => - executorCores = value - parseOpts(tail) - - case ("--executor-memory") :: value :: tail => - executorMemory = value - parseOpts(tail) - - case ("--driver-memory") :: value :: tail => - driverMemory = value - parseOpts(tail) - - case ("--driver-cores") :: value :: tail => - driverCores = value - parseOpts(tail) - - case ("--driver-class-path") :: value :: tail => - driverExtraClassPath = value - parseOpts(tail) - - case ("--driver-java-options") :: value :: tail => - driverExtraJavaOptions = value - parseOpts(tail) - - case ("--driver-library-path") :: value :: tail => - driverExtraLibraryPath = value - parseOpts(tail) - - case ("--properties-file") :: value :: tail => - propertiesFile = value - parseOpts(tail) - - case ("--supervise") :: tail => - supervise = true - parseOpts(tail) - - case ("--queue") :: value :: tail => - queue = value - parseOpts(tail) - - case ("--files") :: value :: tail => - files = value - parseOpts(tail) - - case ("--archives") :: value :: tail => - archives = value - parseOpts(tail) - - case ("--arg") :: value :: tail => - childArgs += value - parseOpts(tail) - - case ("--jars") :: value :: tail => - jars = value - parseOpts(tail) - - case ("--help" | "-h") :: tail => - printUsageAndExit(0) - - case ("--verbose" | "-v") :: tail => - verbose = true - parseOpts(tail) - - case value :: tail => - if (value.startsWith("-")) { - val errMessage = s"Unrecognized option '$value'." - val suggestion: Option[String] = value match { - case v if v.startsWith("--") && v.contains("=") => - val parts = v.split("=") - Some(s"Perhaps you want '${parts(0)} ${parts(1)}'?") - case _ => - None + case ("--class") :: value :: tail => + mainClass = value + parse(tail) + + case ("--deploy-mode") :: value :: tail => + if (value != "client" && value != "cluster") { + SparkSubmit.printErrorAndExit("--deploy-mode must be either \"client\" or \"cluster\"") + } + deployMode = value + parse(tail) + + case ("--num-executors") :: value :: tail => + numExecutors = value + parse(tail) + + case ("--total-executor-cores") :: value :: tail => + totalExecutorCores = value + parse(tail) + + case ("--executor-cores") :: value :: tail => + executorCores = value + parse(tail) + + case ("--executor-memory") :: value :: tail => + executorMemory = value + parse(tail) + + case ("--driver-memory") :: value :: tail => + driverMemory = value + parse(tail) + + case ("--driver-cores") :: value :: tail => + driverCores = value + parse(tail) + + case ("--driver-class-path") :: value :: tail => + driverExtraClassPath = value + parse(tail) + + case ("--driver-java-options") :: value :: tail => + driverExtraJavaOptions = value + parse(tail) + + case ("--driver-library-path") :: value :: tail => + driverExtraLibraryPath = value + parse(tail) + + case ("--properties-file") :: value :: tail => + propertiesFile = value + parse(tail) + + case ("--supervise") :: tail => + supervise = true + parse(tail) + + case ("--queue") :: value :: tail => + queue = value + parse(tail) + + case ("--files") :: value :: tail => + files = value + parse(tail) + + case ("--archives") :: value :: tail => + archives = value + parse(tail) + + case ("--jars") :: value :: tail => + jars = value + parse(tail) + + case ("--help" | "-h") :: tail => + printUsageAndExit(0) + + case ("--verbose" | "-v") :: tail => + verbose = true + parse(tail) + + case value :: tail => + if (inSparkOpts) { + value match { + // convert --foo=bar to --foo bar + case v if v.startsWith("--") && v.contains("=") && v.split("=").size == 2 => + val parts = v.split("=") + parse(Seq(parts(0), parts(1)) ++ tail) + case v if v.startsWith("-") => + val errMessage = s"Unrecognized option '$value'." + SparkSubmit.printErrorAndExit(errMessage) + case v => + primaryResource = v + inSparkOpts = false + parse(tail) + } + } else { + childArgs += value + parse(tail) } - SparkSubmit.printErrorAndExit(errMessage + suggestion.map(" " + _).getOrElse("")) - } - if (primaryResource != null) { - val error = s"Found two conflicting resources, $value and $primaryResource." + - " Expecting only one resource." - SparkSubmit.printErrorAndExit(error) + case Nil => } - primaryResource = value - parseOpts(tail) - - case Nil => } private def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { @@ -277,7 +279,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { outStream.println("Unknown/unsupported param " + unknownParam) } outStream.println( - """Usage: spark-submit [options] + """Usage: spark-submit [options] [app options] |Options: | --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. | --deploy-mode DEPLOY_MODE Mode to deploy the app in, either 'client' or 'cluster'. diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 8351f7156a5e4..5a55e7df34832 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1056,4 +1056,11 @@ private[spark] object Utils extends Logging { def getHadoopFileSystem(path: String): FileSystem = { getHadoopFileSystem(new URI(path)) } + + /** + * Indicates whether Spark is currently running unit tests. + */ + private[spark] def isTesting = { + sys.env.contains("SPARK_TESTING") || sys.props.contains("spark.testing") + } } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 657b44668d385..10a65c75cc621 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -28,6 +28,9 @@ import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers class SparkSubmitSuite extends FunSuite with ShouldMatchers { + def beforeAll() { + System.setProperty("spark.testing", "true") + } val noOpOutputStream = new OutputStream { def write(b: Int) = {} @@ -74,33 +77,35 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { testPrematureExit(Array("--help"), "Usage: spark-submit") } - test("prints error with unrecognized option") { + test("prints error with unrecognized options") { testPrematureExit(Array("--blarg"), "Unrecognized option '--blarg'") testPrematureExit(Array("-bleg"), "Unrecognized option '-bleg'") - testPrematureExit(Array("--master=abc"), - "Unrecognized option '--master=abc'. Perhaps you want '--master abc'?") } - test("handles multiple binary definitions") { - val adjacentJars = Array("foo.jar", "bar.jar") - testPrematureExit(adjacentJars, "error: Found two conflicting resources") + test("handle binary specified but not class") { + testPrematureExit(Array("foo.jar"), "Must specify a main class") + } - val nonAdjacentJars = - Array("foo.jar", "--master", "123", "--class", "abc", "bar.jar") - testPrematureExit(nonAdjacentJars, "error: Found two conflicting resources") + test("handles arguments with --key=val") { + val clArgs = Seq("--jars=one.jar,two.jar,three.jar", "--name=myApp") + val appArgs = new SparkSubmitArguments(clArgs) + appArgs.jars should be ("one.jar,two.jar,three.jar") + appArgs.name should be ("myApp") } - test("handle binary specified but not class") { - testPrematureExit(Array("foo.jar"), "Must specify a main class") + test("handles arguments to user program") { + val clArgs = Seq("--name", "myApp", "userjar.jar", "some", "--random", "args", "here") + val appArgs = new SparkSubmitArguments(clArgs) + appArgs.childArgs should be (Seq("some", "--random", "args", "here")) } test("handles YARN cluster mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "cluster", + val clArgs = Seq("--deploy-mode", "cluster", "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", - "--arg", "arg1", "--arg", "arg2", "--driver-memory", "4g", - "--queue", "thequeue", "--files", "file1.txt,file2.txt", - "--archives", "archive1.txt,archive2.txt", "--num-executors", "6") + "--driver-memory", "4g", "--queue", "thequeue", "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", + "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") @@ -121,12 +126,12 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles YARN client mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "client", + val clArgs = Seq("--deploy-mode", "client", "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", - "--arg", "arg1", "--arg", "arg2", "--driver-memory", "4g", - "--queue", "thequeue", "--files", "file1.txt,file2.txt", - "--archives", "archive1.txt,archive2.txt", "--num-executors", "6") + "--driver-memory", "4g", "--queue", "thequeue", "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -144,9 +149,9 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles standalone cluster mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "cluster", - "--master", "spark://h:p", "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", - "--supervise", "--driver-memory", "4g", "--driver-cores", "5") + val clArgs = Seq("--deploy-mode", "cluster", + "--master", "spark://h:p", "--class", "org.SomeClass", + "--supervise", "--driver-memory", "4g", "--driver-cores", "5", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") @@ -158,10 +163,9 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles standalone client mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "client", + val clArgs = Seq("--deploy-mode", "client", "--master", "spark://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", - "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", - "--driver-memory", "4g") + "--class", "org.SomeClass", "--driver-memory", "4g", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -172,10 +176,9 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles mesos client mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "client", + val clArgs = Seq("--deploy-mode", "client", "--master", "mesos://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", - "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", - "--driver-memory", "4g") + "--class", "org.SomeClass", "--driver-memory", "4g", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -187,22 +190,24 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { test("launch simple application with spark-submit") { runSparkSubmit( - Seq("unUsed.jar", + Seq( "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), "--name", "testApp", - "--master", "local")) + "--master", "local", + "unUsed.jar")) } test("spark submit includes jars passed in through --jar") { val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) val jarsString = Seq(jar1, jar2).map(j => j.toString).mkString(",") - runSparkSubmit( - Seq("unUsed.jar", - "--class", JarCreationTest.getClass.getName.stripSuffix("$"), - "--name", "testApp", - "--master", "local-cluster[2,1,512]", - "--jars", jarsString)) + val args = Seq( + "--class", JarCreationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local-cluster[2,1,512]", + "--jars", jarsString, + "unused.jar") + runSparkSubmit(args) } // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index dcc063042628c..b011679fede2d 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -73,30 +73,34 @@ the bin directory. This script takes care of setting up the classpath with Spark dependencies, and can support different cluster managers and deploy modes that Spark supports. It's usage is - ./bin/spark-submit --class path.to.your.Class [other options..] + ./bin/spark-submit --class path.to.your.Class [options] [app options] -To enumerate all options available to `spark-submit` run it with the `--help` flag. -Here are a few examples of common options: +When calling `spark-submit`, `[app options]` will be passed along to your application's +main class. To enumerate all options available to `spark-submit` run it with +the `--help` flag. Here are a few examples of common options: {% highlight bash %} # Run application locally -./bin/spark-submit my-app.jar \ +./bin/spark-submit \ --class my.main.ClassName - --master local[8] + --master local[8] \ + my-app.jar # Run on a Spark cluster -./bin/spark-submit my-app.jar \ +./bin/spark-submit \ --class my.main.ClassName --master spark://mycluster:7077 \ --executor-memory 20G \ - --total-executor-cores 100 + --total-executor-cores 100 \ + my-app.jar # Run on a YARN cluster -HADOOP_CONF_DIR=XX /bin/spark-submit my-app.jar \ +HADOOP_CONF_DIR=XX /bin/spark-submit \ --class my.main.ClassName --master yarn-cluster \ # can also be `yarn-client` for client mode --executor-memory 20G \ - --num-executors 50 + --num-executors 50 \ + my-app.jar {% endhighlight %} ### Loading Configurations from a File diff --git a/docs/quick-start.md b/docs/quick-start.md index 68afa6e1bff95..64996b52e0404 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -179,9 +179,10 @@ $ sbt package [info] Packaging {..}/{..}/target/scala-2.10/simple-project_2.10-1.0.jar # Use spark-submit to run your application -$ YOUR_SPARK_HOME/bin/spark-submit target/scala-2.10/simple-project_2.10-1.0.jar \ +$ YOUR_SPARK_HOME/bin/spark-submit \ --class "SimpleApp" \ - --master local[4] + --master local[4] \ + target/scala-2.10/simple-project_2.10-1.0.jar ... Lines with a: 46, Lines with b: 23 {% endhighlight %} @@ -272,9 +273,10 @@ $ mvn package [INFO] Building jar: {..}/{..}/target/simple-project-1.0.jar # Use spark-submit to run your application -$ YOUR_SPARK_HOME/bin/spark-submit target/simple-project-1.0.jar \ +$ YOUR_SPARK_HOME/bin/spark-submit \ --class "SimpleApp" \ - --master local[4] + --master local[4] \ + target/simple-project-1.0.jar ... Lines with a: 46, Lines with b: 23 {% endhighlight %} From 7b2527d74deac9512f8ee9ad6d4b060f05e1ab26 Mon Sep 17 00:00:00 2001 From: Rahul Singhal Date: Sun, 27 Apr 2014 15:17:06 -0700 Subject: [PATCH 190/641] SPARK-1650: Correctly identify maven project version Better account for various side-effect outputs while executing "mvn help:evaluate -Dexpression=project.version" Author: Rahul Singhal Closes #572 from rahulsinghaliitd/SPARK-1650 and squashes the following commits: fd6a611 [Rahul Singhal] SPARK-1650: Correctly identify maven project version --- make-distribution.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/make-distribution.sh b/make-distribution.sh index 55fe6c09d0d66..661d1ff5e54c9 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -43,7 +43,7 @@ FWDIR="$(cd `dirname $0`; pwd)" DISTDIR="$FWDIR/dist" -VERSION=$(mvn help:evaluate -Dexpression=project.version |grep -v "INFO") +VERSION=$(mvn help:evaluate -Dexpression=project.version | grep -v "INFO" | tail -n 1) if [ $? == -1 ] ;then echo -e "You need Maven installed to build Spark." echo -e "Download Maven from https://maven.apache.org." From fe65beeaab737a1c9922b84452ca661b39075cfa Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 27 Apr 2014 15:41:57 -0700 Subject: [PATCH 191/641] SPARK-1648 Support closing JIRA's as part of merge script. Adds an automated hook in the merge script that can close the JIRA, set the fix versions, and leave a comment on the JIRA indicating the PR in which it was resolved. This ensures that (a) we always close JIRA's when issues are merged and (b) there is a link to the pull request in every JIRA. This requires a python library called `jira-client`. We could look at embedding this library in our project, but it seemed simple enough to just gracefully disable this feature if it is not installed. It can be installed with `pip install jira-client`. Author: Patrick Wendell Closes #570 from pwendell/jira-pr-merge and squashes the following commits: 3022b96 [Patrick Wendell] SPARK-1648 Support closing JIRA's as part of merge script. --- dev/merge_spark_pr.py | 114 ++++++++++++++++++++++++++++++++++++++---- 1 file changed, 105 insertions(+), 9 deletions(-) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 7a61943e94814..8d15c05d50adc 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -26,19 +26,33 @@ import json import os +import re import subprocess import sys import tempfile import urllib2 +try: + import jira.client + JIRA_IMPORTED=True +except ImportError: + JIRA_IMPORTED=False + # Location of your Spark git development area SPARK_HOME = os.environ.get("SPARK_HOME", "/home/patrick/Documents/spark") # Remote name which points to the Gihub site PR_REMOTE_NAME = os.environ.get("PR_REMOTE_NAME", "apache-github") # Remote name which points to Apache git PUSH_REMOTE_NAME = os.environ.get("PUSH_REMOTE_NAME", "apache") - -GIT_API_BASE = "https://api.github.com/repos/apache/spark" +# ASF JIRA username +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "pwendell") +# ASF JIRA password +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "1234") + +GITHUB_BASE = "https://github.com/apache/spark/pull" +GITHUB_API_BASE = "https://api.github.com/repos/apache/spark" +JIRA_BASE = "https://issues.apache.org/jira/browse" +JIRA_API_BASE = "https://issues.apache.org/jira" # Prefix added to temporary branches BRANCH_PREFIX = "PR_TOOL" @@ -145,8 +159,7 @@ def merge_pr(pr_num, target_ref): return merge_hash -def maybe_cherry_pick(pr_num, merge_hash, default_branch): - continue_maybe("Would you like to pick %s into another branch?" % merge_hash) +def cherry_pick(pr_num, merge_hash, default_branch): pick_ref = raw_input("Enter a branch name [%s]: " % default_branch) if pick_ref == "": pick_ref = default_branch @@ -171,14 +184,86 @@ def maybe_cherry_pick(pr_num, merge_hash, default_branch): print("Pull request #%s picked into %s!" % (pr_num, pick_ref)) print("Pick hash: %s" % pick_hash) + return pick_ref + +def fix_version_from_branch(branch, versions): + # Note: Assumes this is a sorted (newest->oldest) list of un-released versions + if branch == "master": + return versions[0] + else: + branch_ver = branch.replace("branch-", "") + return filter(lambda x: x.name.startswith(branch_ver), versions)[-1] + +def resolve_jira(title, merge_branches, comment): + asf_jira = jira.client.JIRA({'server': JIRA_API_BASE}, + basic_auth=(JIRA_USERNAME, JIRA_PASSWORD)) + + default_jira_id = "" + search = re.findall("SPARK-[0-9]{4,5}", title) + if len(search) > 0: + default_jira_id = search[0] + + jira_id = raw_input("Enter a JIRA id [%s]: " % default_jira_id) + if jira_id == "": + jira_id = default_jira_id -branches = get_json("%s/branches" % GIT_API_BASE) + try: + issue = asf_jira.issue(jira_id) + except Exception as e: + fail("ASF JIRA could not find %s\n%s" % (jira_id, e)) + + cur_status = issue.fields.status.name + cur_summary = issue.fields.summary + cur_assignee = issue.fields.assignee + if cur_assignee == None: + cur_assignee = "NOT ASSIGNED!!!" + else: + cur_assignee = cur_assignee.displayName + + if cur_status == "Resolved" or cur_status == "Closed": + fail("JIRA issue %s already has status '%s'" % (jira_id, cur_status)) + print ("=== JIRA %s ===" % jira_id) + print ("summary\t\t%s\nassignee\t%s\nstatus\t\t%s\nurl\t\t%s/%s\n" % ( + cur_summary, cur_assignee, cur_status, JIRA_BASE, jira_id)) + + versions = asf_jira.project_versions("SPARK") + versions = sorted(versions, key = lambda x: x.name, reverse=True) + versions = filter(lambda x: x.raw['released'] == False, versions) + + default_fix_versions = map(lambda x: fix_version_from_branch(x, versions).name, merge_branches) + for v in default_fix_versions: + # Handles the case where we have forked a release branch but not yet made the release. + # In this case, if the PR is committed to the master branch and the release branch, we + # only consider the release branch to be the fix version. E.g. it is not valid to have + # both 1.1.0 and 1.0.0 as fix versions. + (major, minor, patch) = v.split(".") + if patch == 0: + previous = "%s.%s.%s" % (major, int(minor) - 1, 0) + if previous in default_fix_versions: + default_fix_versions = filter(lambda x: x != v, default_fix_versions) + default_fix_versions = ",".join(default_fix_versions) + + fix_versions = raw_input("Enter comma-separated fix version(s) [%s]: " % default_fix_versions) + if fix_versions == "": + fix_versions = default_fix_versions + fix_versions = fix_versions.replace(" ", "").split(",") + + def get_version_json(version_str): + return filter(lambda v: v.name == version_str, versions)[0].raw + jira_fix_versions = map(lambda v: get_version_json(v), fix_versions) + + resolve = filter(lambda a: a['name'] == "Resolve Issue", asf_jira.transitions(jira_id))[0] + asf_jira.transition_issue(jira_id, resolve["id"], fixVersions=jira_fix_versions, comment=comment) + + print "Succesfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions) + +branches = get_json("%s/branches" % GITHUB_API_BASE) branch_names = filter(lambda x: x.startswith("branch-"), [x['name'] for x in branches]) # Assumes branch names can be sorted lexicographically latest_branch = sorted(branch_names, reverse=True)[0] pr_num = raw_input("Which pull request would you like to merge? (e.g. 34): ") -pr = get_json("%s/pulls/%s" % (GIT_API_BASE, pr_num)) +pr = get_json("%s/pulls/%s" % (GITHUB_API_BASE, pr_num)) url = pr["url"] title = pr["title"] @@ -208,11 +293,22 @@ def maybe_cherry_pick(pr_num, merge_hash, default_branch): continue_maybe(msg) print ("\n=== Pull Request #%s ===" % pr_num) -print("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( +print ("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( title, pr_repo_desc, target_ref, url)) continue_maybe("Proceed with merging pull request #%s?" % pr_num) +merged_refs = [target_ref] + merge_hash = merge_pr(pr_num, target_ref) -while True: - maybe_cherry_pick(pr_num, merge_hash, latest_branch) +pick_prompt = "Would you like to pick %s into another branch?" % merge_hash +while raw_input("\n%s (y/n): " % pick_prompt).lower() == "y": + merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)] + +if JIRA_IMPORTED: + continue_maybe("Would you like to update an associated JIRA?") + jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) + resolve_jira(title, merged_refs, jira_comment) +else: + print "Could not find jira-python library. Run 'sudo pip install jira-python' to install." + print "Exiting without trying to close the associated JIRA." From eefb90d382747c29d7537630ed5ad2c783bb8263 Mon Sep 17 00:00:00 2001 From: Rahul Singhal Date: Sun, 27 Apr 2014 15:50:48 -0700 Subject: [PATCH 192/641] SPARK-1651: Delete existing deployment directory Small bug fix to make sure the "spark contents" are copied to the deployment directory correctly. Author: Rahul Singhal Closes #573 from rahulsinghaliitd/SPARK-1651 and squashes the following commits: 402c999 [Rahul Singhal] SPARK-1651: Delete existing deployment directory --- make-distribution.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/make-distribution.sh b/make-distribution.sh index 661d1ff5e54c9..4ac80efae0ab2 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -178,6 +178,7 @@ fi if [ "$MAKE_TGZ" == "true" ]; then TARDIR_NAME=spark-$VERSION-bin-$NAME TARDIR="$FWDIR/$TARDIR_NAME" + rm -rf "$TARDIR" cp -r "$DISTDIR" "$TARDIR" tar czf "spark-$VERSION-bin-$NAME.tgz" -C "$FWDIR" "$TARDIR_NAME" rm -rf "$TARDIR" From 3d9fb09681308abd2066d0d02f2438f5a17c9dd9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 27 Apr 2014 15:45:17 -0700 Subject: [PATCH 193/641] HOTFIX: Minor patch to merge script. --- dev/merge_spark_pr.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 8d15c05d50adc..83618c8068d35 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -237,7 +237,7 @@ def resolve_jira(title, merge_branches, comment): # only consider the release branch to be the fix version. E.g. it is not valid to have # both 1.1.0 and 1.0.0 as fix versions. (major, minor, patch) = v.split(".") - if patch == 0: + if patch == "0": previous = "%s.%s.%s" % (major, int(minor) - 1, 0) if previous in default_fix_versions: default_fix_versions = filter(lambda x: x != v, default_fix_versions) From 6b3c6e5dd8e74435f71ecdb224db532550ef407b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 27 Apr 2014 17:40:56 -0700 Subject: [PATCH 194/641] SPARK-1145: Memory mapping with many small blocks can cause JVM allocation failures This includes some minor code clean-up as well. The main change is that small files are not memory mapped. There is a nicer way to write that code block using Scala's `Try` but to make it easy to back port and as simple as possible, I opted for the more explicit but less pretty format. Author: Patrick Wendell Closes #43 from pwendell/block-iter-logging and squashes the following commits: 1cff512 [Patrick Wendell] Small issue from merge. 49f6c269 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into block-iter-logging 4943351 [Patrick Wendell] Added a test and feedback on mateis review a637a18 [Patrick Wendell] Review feedback and adding rewind() when reading byte buffers. b76b95f [Patrick Wendell] Review feedback 4e1514e [Patrick Wendell] Don't memory map for small files d238b88 [Patrick Wendell] Some logging and clean-up --- .../spark/storage/BlockFetcherIterator.scala | 22 +++---- .../apache/spark/storage/BlockManager.scala | 3 +- .../org/apache/spark/storage/DiskStore.scala | 16 ++++- .../scala/org/apache/spark/util/Utils.scala | 3 +- .../spark/storage/BlockManagerSuite.scala | 58 +++++++++++++++++-- docs/configuration.md | 9 +++ 6 files changed, 91 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index ace9cd51c96b7..a02dd9441d679 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -148,6 +148,12 @@ object BlockFetcherIterator { } protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { + // Make remote requests at most maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val targetRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize) + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. val remoteRequests = new ArrayBuffer[FetchRequest] @@ -159,11 +165,6 @@ object BlockFetcherIterator { _numBlocksToFetch += localBlocksToFetch.size } else { numRemote += blockInfos.size - // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val minRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) val iterator = blockInfos.iterator var curRequestSize = 0L var curBlocks = new ArrayBuffer[(BlockId, Long)] @@ -178,11 +179,12 @@ object BlockFetcherIterator { } else if (size < 0) { throw new BlockException(blockId, "Negative block size " + size) } - if (curRequestSize >= minRequestSize) { + if (curRequestSize >= targetRequestSize) { // Add this FetchRequest remoteRequests += new FetchRequest(address, curBlocks) curRequestSize = 0 curBlocks = new ArrayBuffer[(BlockId, Long)] + logDebug(s"Creating fetch request of $curRequestSize at $address") } } // Add in the final request @@ -191,7 +193,7 @@ object BlockFetcherIterator { } } } - logInfo("Getting " + _numBlocksToFetch + " non-zero-bytes blocks out of " + + logInfo("Getting " + _numBlocksToFetch + " non-empty blocks out of " + totalBlocks + " blocks") remoteRequests } @@ -226,8 +228,8 @@ object BlockFetcherIterator { sendRequest(fetchRequests.dequeue()) } - val numGets = remoteRequests.size - fetchRequests.size - logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) + val numFetches = remoteRequests.size - fetchRequests.size + logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime)) // Get Local Blocks startTime = System.currentTimeMillis @@ -327,7 +329,7 @@ object BlockFetcherIterator { } copiers = startCopiers(conf.getInt("spark.shuffle.copier.threads", 6)) - logInfo("Started " + fetchRequestsSync.size + " remote gets in " + + logInfo("Started " + fetchRequestsSync.size + " remote fetches in " + Utils.getUsedTimeMs(startTime)) // Get Local Blocks diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 02ba5ecf52459..6d7d4f922e1fa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -46,11 +46,12 @@ private[spark] class BlockManager( val master: BlockManagerMaster, val defaultSerializer: Serializer, maxMemory: Long, - val conf: SparkConf, + val _conf: SparkConf, securityManager: SecurityManager, mapOutputTracker: MapOutputTracker) extends Logging { + def conf = _conf val shuffleBlockManager = new ShuffleBlockManager(this) val diskBlockManager = new DiskBlockManager(shuffleBlockManager, conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 36ee4bcc41c66..0ab9fad422717 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -33,6 +33,8 @@ import org.apache.spark.util.Utils private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) extends BlockStore(blockManager) with Logging { + val minMemoryMapBytes = blockManager.conf.getLong("spark.storage.memoryMapThreshold", 2 * 4096L) + override def getSize(blockId: BlockId): Long = { diskManager.getBlockLocation(blockId).length } @@ -94,12 +96,20 @@ private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManage override def getBytes(blockId: BlockId): Option[ByteBuffer] = { val segment = diskManager.getBlockLocation(blockId) val channel = new RandomAccessFile(segment.file, "r").getChannel() - val buffer = try { - channel.map(MapMode.READ_ONLY, segment.offset, segment.length) + + try { + // For small files, directly read rather than memory map + if (segment.length < minMemoryMapBytes) { + val buf = ByteBuffer.allocate(segment.length.toInt) + channel.read(buf, segment.offset) + buf.flip() + Some(buf) + } else { + Some(channel.map(MapMode.READ_ONLY, segment.offset, segment.length)) + } } finally { channel.close() } - Some(buffer) } override def getValues(blockId: BlockId): Option[Iterator[Any]] = { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 5a55e7df34832..b678604ff81c8 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -553,8 +553,7 @@ private[spark] object Utils extends Logging { } /** - * Return the string to tell how long has passed in seconds. The passing parameter should be in - * millisecond. + * Return the string to tell how long has passed in milliseconds. */ def getUsedTimeMs(startTimeMs: Long): String = { " " + (System.currentTimeMillis - startTimeMs) + " ms" diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 907428db80af3..00deecc1c3ca9 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -17,12 +17,15 @@ package org.apache.spark.storage -import java.nio.ByteBuffer +import java.nio.{ByteBuffer, MappedByteBuffer} +import java.util.Arrays import akka.actor._ -import org.scalatest.BeforeAndAfter -import org.scalatest.FunSuite -import org.scalatest.PrivateMethodTester +import org.apache.spark.SparkConf +import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} +import org.mockito.Mockito.{mock, when} +import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers._ @@ -785,6 +788,53 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } } + test("reads of memory-mapped and non memory-mapped files are equivalent") { + val confKey = "spark.storage.memoryMapThreshold" + + // Create a non-trivial (not all zeros) byte array + var counter = 0.toByte + def incr = {counter = (counter + 1).toByte; counter;} + val bytes = Array.fill[Byte](1000)(incr) + val byteBuffer = ByteBuffer.wrap(bytes) + + val blockId = BlockId("rdd_1_2") + + // This sequence of mocks makes these tests fairly brittle. It would + // be nice to refactor classes involved in disk storage in a way that + // allows for easier testing. + val blockManager = mock(classOf[BlockManager]) + val shuffleBlockManager = mock(classOf[ShuffleBlockManager]) + when(shuffleBlockManager.conf).thenReturn(conf) + val diskBlockManager = new DiskBlockManager(shuffleBlockManager, + System.getProperty("java.io.tmpdir")) + + when(blockManager.conf).thenReturn(conf.clone.set(confKey, 0.toString)) + val diskStoreMapped = new DiskStore(blockManager, diskBlockManager) + diskStoreMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY) + val mapped = diskStoreMapped.getBytes(blockId).get + + when(blockManager.conf).thenReturn(conf.clone.set(confKey, (1000 * 1000).toString)) + val diskStoreNotMapped = new DiskStore(blockManager, diskBlockManager) + diskStoreNotMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY) + val notMapped = diskStoreNotMapped.getBytes(blockId).get + + // Not possible to do isInstanceOf due to visibility of HeapByteBuffer + assert(notMapped.getClass.getName.endsWith("HeapByteBuffer"), + "Expected HeapByteBuffer for un-mapped read") + assert(mapped.isInstanceOf[MappedByteBuffer], "Expected MappedByteBuffer for mapped read") + + def arrayFromByteBuffer(in: ByteBuffer): Array[Byte] = { + val array = new Array[Byte](in.remaining()) + in.get(array) + array + } + + val mappedAsArray = arrayFromByteBuffer(mapped) + val notMappedAsArray = arrayFromByteBuffer(notMapped) + assert(Arrays.equals(mappedAsArray, bytes)) + assert(Arrays.equals(notMappedAsArray, bytes)) + } + test("updated block statuses") { store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr, mapOutputTracker) diff --git a/docs/configuration.md b/docs/configuration.md index 8d3442625b475..b078c7c1112c2 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -131,6 +131,15 @@ Apart from these, the following properties are also available, and may be useful spark.storage.memoryFraction. + + spark.storage.memoryMapThreshold + 8192 + + Size of a block, in bytes, above which Spark memory maps when reading a block from disk. + This prevents Spark from memory mapping very small blocks. In general, memory + mapping has high overhead for blocks close to or below the page size of the operating system. + + spark.tachyonStore.baseDir System.getProperty("java.io.tmpdir") From 71f4d2612a1be2904ed1536280680abc2dd212e7 Mon Sep 17 00:00:00 2001 From: witgo Date: Sun, 27 Apr 2014 19:41:02 -0700 Subject: [PATCH 195/641] Fix SPARK-1609: Executor fails to start when Command.extraJavaOptions contains multiple Java options Author: witgo Closes #547 from witgo/SPARK-1609 and squashes the following commits: deb6a4c [witgo] review commit 91da0bb [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1609 0640852 [witgo] review commit 8f90b22 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1609 bcf36cb [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1609 1185605 [witgo] fix extraJavaOptions split f7c0ab7 [witgo] bugfix 86fc4bb [witgo] bugfix 8a265b7 [witgo] Fix SPARK-1609: Executor fails to start when use spark-submit --- .../org/apache/spark/deploy/worker/CommandUtils.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 9103c885fa96c..3e615e753b342 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -48,7 +48,8 @@ object CommandUtils extends Logging { def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") // Note, this will coalesce multiple options into a single command component - val extraOpts = command.extraJavaOptions.toSeq + val extraOpts = command.extraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq()) + val libraryOpts = if (command.libraryPathEntries.size > 0) { val joined = command.libraryPathEntries.mkString(File.pathSeparator) @@ -62,10 +63,10 @@ object CommandUtils extends Logging { val classPath = Utils.executeAndGetOutput( Seq(sparkHome + "/bin/compute-classpath" + ext), extraEnvironment=command.environment) - val userClassPath = command.classPathEntries.mkString(File.pathSeparator) - val classPathWithUser = classPath + File.pathSeparator + userClassPath + val userClassPath = command.classPathEntries ++ Seq(classPath) - Seq("-cp", classPathWithUser) ++ libraryOpts ++ extraOpts ++ memoryOpts + Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++ + libraryOpts ++ extraOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ From ea01affc34a8b816f92a71e0f8f6901397c8a4a3 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Sun, 27 Apr 2014 23:57:29 -0700 Subject: [PATCH 196/641] Update the import package name for TestHive in sbt shell sbt/sbt hive/console will fail as TestHive changed its package from "org.apache.spark.sql.hive" to "org.apache.spark.sql.hive.test". Author: Cheng Hao Closes #574 from chenghao-intel/hive_console and squashes the following commits: de14035 [Cheng Hao] Update the import package name for TestHive in sbt shell --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 48f234ce87a44..5adfbe373cb00 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -506,7 +506,7 @@ object SparkBuild extends Build { |import org.apache.spark.sql.catalyst.util._ |import org.apache.spark.sql.execution |import org.apache.spark.sql.hive._ - |import org.apache.spark.sql.hive.TestHive._ + |import org.apache.spark.sql.hive.test.TestHive._ |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin ) From f735884414a15c0c07df60068ee11f9da47eff77 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Sun, 27 Apr 2014 23:59:42 -0700 Subject: [PATCH 197/641] [SQL]Append some missing types for HiveUDF Add the missing types Author: Cheng Hao Closes #459 from chenghao-intel/missing_types and squashes the following commits: 21cba2e [Cheng Hao] Append some missing types for HiveUDF --- .../org/apache/spark/sql/hive/hiveUdfs.scala | 58 +++++++++++++++---- 1 file changed, 48 insertions(+), 10 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index a09270eb7b134..c7de4ab6d3955 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -70,24 +70,26 @@ private[hive] object HiveFunctionRegistry } def javaClassToDataType(clz: Class[_]): DataType = clz match { + // writable case c: Class[_] if c == classOf[hadoopIo.DoubleWritable] => DoubleType case c: Class[_] if c == classOf[hiveIo.DoubleWritable] => DoubleType case c: Class[_] if c == classOf[hiveIo.HiveDecimalWritable] => DecimalType case c: Class[_] if c == classOf[hiveIo.ByteWritable] => ByteType case c: Class[_] if c == classOf[hiveIo.ShortWritable] => ShortType + case c: Class[_] if c == classOf[hiveIo.TimestampWritable] => TimestampType case c: Class[_] if c == classOf[hadoopIo.Text] => StringType case c: Class[_] if c == classOf[hadoopIo.IntWritable] => IntegerType case c: Class[_] if c == classOf[hadoopIo.LongWritable] => LongType case c: Class[_] if c == classOf[hadoopIo.FloatWritable] => FloatType case c: Class[_] if c == classOf[hadoopIo.BooleanWritable] => BooleanType + case c: Class[_] if c == classOf[hadoopIo.BytesWritable] => BinaryType + + // java class case c: Class[_] if c == classOf[java.lang.String] => StringType - case c: Class[_] if c == java.lang.Short.TYPE => ShortType - case c: Class[_] if c == java.lang.Integer.TYPE => IntegerType - case c: Class[_] if c == java.lang.Long.TYPE => LongType - case c: Class[_] if c == java.lang.Double.TYPE => DoubleType - case c: Class[_] if c == java.lang.Byte.TYPE => ByteType - case c: Class[_] if c == java.lang.Float.TYPE => FloatType - case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType + case c: Class[_] if c == classOf[java.sql.Timestamp] => TimestampType + case c: Class[_] if c == classOf[HiveDecimal] => DecimalType + case c: Class[_] if c == classOf[java.math.BigDecimal] => DecimalType + case c: Class[_] if c == classOf[Array[Byte]] => BinaryType case c: Class[_] if c == classOf[java.lang.Short] => ShortType case c: Class[_] if c == classOf[java.lang.Integer] => IntegerType case c: Class[_] if c == classOf[java.lang.Long] => LongType @@ -95,6 +97,16 @@ private[hive] object HiveFunctionRegistry case c: Class[_] if c == classOf[java.lang.Byte] => ByteType case c: Class[_] if c == classOf[java.lang.Float] => FloatType case c: Class[_] if c == classOf[java.lang.Boolean] => BooleanType + + // primitive type + case c: Class[_] if c == java.lang.Short.TYPE => ShortType + case c: Class[_] if c == java.lang.Integer.TYPE => IntegerType + case c: Class[_] if c == java.lang.Long.TYPE => LongType + case c: Class[_] if c == java.lang.Double.TYPE => DoubleType + case c: Class[_] if c == java.lang.Byte.TYPE => ByteType + case c: Class[_] if c == java.lang.Float.TYPE => FloatType + case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType + case c: Class[_] if c.isArray => ArrayType(javaClassToDataType(c.getComponentType)) } } @@ -111,11 +123,19 @@ private[hive] trait HiveFunctionFactory { case i: hadoopIo.IntWritable => i.get case t: hadoopIo.Text => t.toString case l: hadoopIo.LongWritable => l.get - case d: hadoopIo.DoubleWritable => d.get() + case d: hadoopIo.DoubleWritable => d.get case d: hiveIo.DoubleWritable => d.get case s: hiveIo.ShortWritable => s.get - case b: hadoopIo.BooleanWritable => b.get() + case b: hadoopIo.BooleanWritable => b.get case b: hiveIo.ByteWritable => b.get + case b: hadoopIo.FloatWritable => b.get + case b: hadoopIo.BytesWritable => { + val bytes = new Array[Byte](b.getLength) + System.arraycopy(b.getBytes(), 0, bytes, 0, b.getLength) + bytes + } + case t: hiveIo.TimestampWritable => t.getTimestamp + case b: hiveIo.HiveDecimalWritable => BigDecimal(b.getHiveDecimal().bigDecimalValue()) case list: java.util.List[_] => list.map(unwrap) case map: java.util.Map[_,_] => map.map { case (k, v) => (unwrap(k), unwrap(v)) }.toMap case array: Array[_] => array.map(unwrap).toSeq @@ -127,6 +147,9 @@ private[hive] trait HiveFunctionFactory { case p: java.lang.Byte => p case p: java.lang.Boolean => p case str: String => str + case p: BigDecimal => p + case p: Array[Byte] => p + case p: java.sql.Timestamp => p } } @@ -252,13 +275,17 @@ private[hive] trait HiveInspectors { /** Converts native catalyst types to the types expected by Hive */ def wrap(a: Any): AnyRef = a match { - case s: String => new hadoopIo.Text(s) + case s: String => new hadoopIo.Text(s) // TODO why should be Text? case i: Int => i: java.lang.Integer case b: Boolean => b: java.lang.Boolean + case f: Float => f: java.lang.Float case d: Double => d: java.lang.Double case l: Long => l: java.lang.Long case l: Short => l: java.lang.Short case l: Byte => l: java.lang.Byte + case b: BigDecimal => b.bigDecimal + case b: Array[Byte] => b + case t: java.sql.Timestamp => t case s: Seq[_] => seqAsJavaList(s.map(wrap)) case m: Map[_,_] => mapAsJavaMap(m.map { case (k, v) => wrap(k) -> wrap(v) }) @@ -280,6 +307,8 @@ private[hive] trait HiveInspectors { case ByteType => PrimitiveObjectInspectorFactory.javaByteObjectInspector case NullType => PrimitiveObjectInspectorFactory.javaVoidObjectInspector case BinaryType => PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector + case TimestampType => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector + case DecimalType => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector } def inspectorToDataType(inspector: ObjectInspector): DataType = inspector match { @@ -307,6 +336,14 @@ private[hive] trait HiveInspectors { case _: JavaShortObjectInspector => ShortType case _: WritableByteObjectInspector => ByteType case _: JavaByteObjectInspector => ByteType + case _: WritableFloatObjectInspector => FloatType + case _: JavaFloatObjectInspector => FloatType + case _: WritableBinaryObjectInspector => BinaryType + case _: JavaBinaryObjectInspector => BinaryType + case _: WritableHiveDecimalObjectInspector => DecimalType + case _: JavaHiveDecimalObjectInspector => DecimalType + case _: WritableTimestampObjectInspector => TimestampType + case _: JavaTimestampObjectInspector => TimestampType } implicit class typeInfoConversions(dt: DataType) { @@ -324,6 +361,7 @@ private[hive] trait HiveInspectors { case ShortType => shortTypeInfo case StringType => stringTypeInfo case DecimalType => decimalTypeInfo + case TimestampType => timestampTypeInfo case NullType => voidTypeInfo } } From 1d84964bf80f4e69e54d62286c3861c2362342d0 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 28 Apr 2014 13:58:09 -0700 Subject: [PATCH 198/641] [SPARK-1633][Streaming] Java API unit test and example for custom streaming receiver in Java Author: Tathagata Das Closes #558 from tdas/more-fixes and squashes the following commits: c0c84e6 [Tathagata Das] Removing extra println() d8a8cf4 [Tathagata Das] More tweaks to make unit test work in Jenkins. b7caa98 [Tathagata Das] More tweaks. d337367 [Tathagata Das] More tweaks 22d6f2d [Tathagata Das] Merge remote-tracking branch 'apache/master' into more-fixes 40a961b [Tathagata Das] Modified java test to reduce flakiness. 9410ca6 [Tathagata Das] Merge remote-tracking branch 'apache/master' into more-fixes 86d9147 [Tathagata Das] scala style fix 2f3d7b1 [Tathagata Das] Added Scala custom receiver example. d677611 [Tathagata Das] Merge remote-tracking branch 'apache/master' into more-fixes bec3fc2 [Tathagata Das] Added license. 51d6514 [Tathagata Das] Fixed docs on receiver. 81aafa0 [Tathagata Das] Added Java test for Receiver API, and added JavaCustomReceiver example. --- .../examples/JavaCustomReceiver.java | 152 ++++++++++++++++++ .../examples/JavaNetworkWordCount.java | 5 +- .../streaming/examples/CustomReceiver.scala | 108 +++++++++++++ .../api/java/JavaStreamingContext.scala | 2 +- .../spark/streaming/receiver/Receiver.scala | 90 ++++++++--- .../spark/streaming/JavaReceiverAPISuite.java | 144 +++++++++++++++++ .../spark/streaming/JavaTestUtils.scala | 4 +- .../spark/streaming/InputStreamsSuite.scala | 3 +- .../streaming/StreamingContextSuite.scala | 2 +- .../spark/streaming/TestSuiteBase.scala | 1 + 10 files changed, 476 insertions(+), 35 deletions(-) create mode 100644 examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java create mode 100644 examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala create mode 100644 streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java new file mode 100644 index 0000000000000..a94fa621dc328 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.examples; + +import com.google.common.collect.Lists; + +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import org.apache.spark.streaming.receiver.Receiver; +import scala.Tuple2; + +import java.io.BufferedReader; +import java.io.InputStreamReader; +import java.net.ConnectException; +import java.net.Socket; +import java.util.regex.Pattern; + +/** + * Custom Receiver that receives data over a socket. Received bytes is interpreted as + * text and \n delimited lines are considered as records. They are then counted and printed. + * + * Usage: JavaCustomReceiver + * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * and of the TCP server that Spark Streaming would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ ./run org.apache.spark.streaming.examples.JavaCustomReceiver local[2] localhost 9999` + */ + +public class JavaCustomReceiver extends Receiver { + private static final Pattern SPACE = Pattern.compile(" "); + + public static void main(String[] args) { + if (args.length < 3) { + System.err.println("Usage: JavaNetworkWordCount \n" + + "In local mode, should be 'local[n]' with n > 1"); + System.exit(1); + } + + StreamingExamples.setStreamingLogLevels(); + + // Create the context with a 1 second batch size + JavaStreamingContext ssc = new JavaStreamingContext(args[0], "JavaNetworkWordCount", + new Duration(1000), System.getenv("SPARK_HOME"), + JavaStreamingContext.jarOfClass(JavaNetworkWordCount.class)); + + // Create a input stream with the custom receiver on target ip:port and count the + // words in input stream of \n delimited text (eg. generated by 'nc') + JavaDStream lines = ssc.receiverStream( + new JavaCustomReceiver(args[1], Integer.parseInt(args[2]))); + JavaDStream words = lines.flatMap(new FlatMapFunction() { + @Override + public Iterable call(String x) { + return Lists.newArrayList(SPACE.split(x)); + } + }); + JavaPairDStream wordCounts = words.mapToPair( + new PairFunction() { + @Override public Tuple2 call(String s) { + return new Tuple2(s, 1); + } + }).reduceByKey(new Function2() { + @Override + public Integer call(Integer i1, Integer i2) { + return i1 + i2; + } + }); + + wordCounts.print(); + ssc.start(); + ssc.awaitTermination(); + } + + // ============= Receiver code that receives data over a socket ============== + + String host = null; + int port = -1; + + public JavaCustomReceiver(String host_ , int port_) { + super(StorageLevel.MEMORY_AND_DISK_2()); + host = host_; + port = port_; + } + + public void onStart() { + // Start the thread that receives data over a connection + new Thread() { + @Override public void run() { + receive(); + } + }.start(); + } + + public void onStop() { + // There is nothing much to do as the thread calling receive() + // is designed to stop by itself isStopped() returns false + } + + /** Create a socket connection and receive data until receiver is stopped */ + private void receive() { + Socket socket = null; + String userInput = null; + + try { + // connect to the server + socket = new Socket(host, port); + + BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream())); + + // Until stopped or connection broken continue reading + while (!isStopped() && (userInput = reader.readLine()) != null) { + System.out.println("Received data '" + userInput + "'"); + store(userInput); + } + reader.close(); + socket.close(); + + // Restart in an attempt to connect again when server is active again + restart("Trying to connect again"); + } catch(ConnectException ce) { + // restart if could not connect to server + restart("Could not connect", ce); + } catch(Throwable t) { + restart("Error receiving data", t); + } + } +} + + diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java index 7f68d451e9b31..0cc9d0ae1a08e 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java @@ -31,7 +31,7 @@ /** * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. - * Usage: NetworkWordCount + * Usage: JavaNetworkWordCount * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. * and describe the TCP server that Spark Streaming would connect to receive data. * @@ -43,9 +43,6 @@ public final class JavaNetworkWordCount { private static final Pattern SPACE = Pattern.compile(" "); - private JavaNetworkWordCount() { - } - public static void main(String[] args) { if (args.length < 3) { System.err.println("Usage: JavaNetworkWordCount \n" + diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala new file mode 100644 index 0000000000000..eebffd824983f --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.examples + +import java.io.{InputStreamReader, BufferedReader, InputStream} +import java.net.Socket + +import org.apache.spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Seconds, StreamingContext} +import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.receiver.Receiver + +/** + * Custom Receiver that receives data over a socket. Received bytes is interpreted as + * text and \n delimited lines are considered as records. They are then counted and printed. + * + * Usage: CustomReceiver + * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * and of the TCP server that Spark Streaming would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ ./run org.apache.spark.streaming.examples.CustomReceiver local[2] localhost 9999` + */ +object CustomReceiver { + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: NetworkWordCount \n" + + "In local mode, should be 'local[n]' with n > 1") + System.exit(1) + } + + StreamingExamples.setStreamingLogLevels() + + // Create the context with a 1 second batch size + val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1), + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + + // Create a input stream with the custom receiver on target ip:port and count the + // words in input stream of \n delimited text (eg. generated by 'nc') + val lines = ssc.receiverStream(new CustomReceiver(args(1), args(2).toInt)) + val words = lines.flatMap(_.split(" ")) + val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) + wordCounts.print() + ssc.start() + ssc.awaitTermination() + } +} + + +class CustomReceiver(host: String, port: Int) + extends Receiver[String](StorageLevel.MEMORY_AND_DISK_2) with Logging { + + def onStart() { + // Start the thread that receives data over a connection + new Thread("Socket Receiver") { + override def run() { receive() } + }.start() + } + + def onStop() { + // There is nothing much to do as the thread calling receive() + // is designed to stop by itself isStopped() returns false + } + + /** Create a socket connection and receive data until receiver is stopped */ + private def receive() { + var socket: Socket = null + var userInput: String = null + try { + logInfo("Connecting to " + host + ":" + port) + socket = new Socket(host, port) + logInfo("Connected to " + host + ":" + port) + val reader = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8")) + userInput = reader.readLine() + while(!isStopped && userInput != null) { + store(userInput) + userInput = reader.readLine() + } + reader.close() + socket.close() + logInfo("Stopped receiving") + restart("Trying to connect again") + } catch { + case e: java.net.ConnectException => + restart("Error connecting to " + host + ":" + port, e) + case t: Throwable => + restart("Error receiving data", t) + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index fbb2e9f85dd12..75a3e9334e6d5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -390,7 +390,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html * @param receiver Custom implementation of Receiver */ - def receiverStream[T](receiver: Receiver[T]): ReceiverInputDStream[T] = { + def receiverStream[T](receiver: Receiver[T]): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.receiverStream(receiver) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala index 524c1b8d8ce46..b310c22b3ab78 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -30,28 +30,55 @@ import org.apache.spark.annotation.DeveloperApi * Abstract class of a receiver that can be run on worker nodes to receive external data. A * custom receiver can be defined by defining the functions onStart() and onStop(). onStart() * should define the setup steps necessary to start receiving data, - * and onStop() should define the cleanup steps necessary to stop receiving data. A custom - * receiver would look something like this. + * and onStop() should define the cleanup steps necessary to stop receiving data. * - * @example {{{ + * A custom receiver in Scala would look like this. + * + * {{{ * class MyReceiver(storageLevel: StorageLevel) extends NetworkReceiver[String](storageLevel) { - * def onStart() { - * // Setup stuff (start threads, open sockets, etc.) to start receiving data. - * // Must start new thread to receive data, as onStart() must be non-blocking. + * def onStart() { + * // Setup stuff (start threads, open sockets, etc.) to start receiving data. + * // Must start new thread to receive data, as onStart() must be non-blocking. * - * // Call store(...) in those threads to store received data into Spark's memory. + * // Call store(...) in those threads to store received data into Spark's memory. * - * // Call stop(...), restart() or reportError(...) on any thread based on how - * // different errors should be handled. + * // Call stop(...), restart(...) or reportError(...) on any thread based on how + * // different errors needs to be handled. * - * // See corresponding method documentation for more details - * } + * // See corresponding method documentation for more details + * } * - * def onStop() { - * // Cleanup stuff (stop threads, close sockets, etc.) to stop receiving data. - * } + * def onStop() { + * // Cleanup stuff (stop threads, close sockets, etc.) to stop receiving data. + * } * } * }}} + * + * A custom receiver in Java would look like this. + * + * {{{ + * class MyReceiver extends Receiver { + * public MyReceiver(StorageLevel storageLevel) { + * super(storageLevel); + * } + * + * public void onStart() { + * // Setup stuff (start threads, open sockets, etc.) to start receiving data. + * // Must start new thread to receive data, as onStart() must be non-blocking. + * + * // Call store(...) in those threads to store received data into Spark's memory. + * + * // Call stop(...), restart(...) or reportError(...) on any thread based on how + * // different errors needs to be handled. + * + * // See corresponding method documentation for more details + * } + * + * public void onStop() { + * // Cleanup stuff (stop threads, close sockets, etc.) to stop receiving data. + * } + * } + * }}} */ @DeveloperApi abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable { @@ -156,30 +183,34 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable } /** - * Restart the receiver. This will call `onStop()` immediately and return. - * Asynchronously, after a delay, `onStart()` will be called. + * Restart the receiver. This method schedules the restart and returns + * immediately. The stopping and subsequent starting of the receiver + * (by calling `onStop()` and `onStart()`) is performed asynchronously + * in a background thread. The delay between the stopping and the starting + * is defined by the Spark configuration `spark.streaming.receiverRestartDelay`. * The `message` will be reported to the driver. - * The delay is defined by the Spark configuration - * `spark.streaming.receiverRestartDelay`. */ def restart(message: String) { executor.restartReceiver(message) } /** - * Restart the receiver. This will call `onStop()` immediately and return. - * Asynchronously, after a delay, `onStart()` will be called. + * Restart the receiver. This method schedules the restart and returns + * immediately. The stopping and subsequent starting of the receiver + * (by calling `onStop()` and `onStart()`) is performed asynchronously + * in a background thread. The delay between the stopping and the starting + * is defined by the Spark configuration `spark.streaming.receiverRestartDelay`. * The `message` and `exception` will be reported to the driver. - * The delay is defined by the Spark configuration - * `spark.streaming.receiverRestartDelay`. */ def restart(message: String, error: Throwable) { executor.restartReceiver(message, Some(error)) } /** - * Restart the receiver. This will call `onStop()` immediately and return. - * Asynchronously, after the given delay, `onStart()` will be called. + * Restart the receiver. This method schedules the restart and returns + * immediately. The stopping and subsequent starting of the receiver + * (by calling `onStop()` and `onStart()`) is performed asynchronously + * in a background thread. */ def restart(message: String, error: Throwable, millisecond: Int) { executor.restartReceiver(message, Some(error), millisecond) @@ -195,16 +226,23 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable executor.stop(message, Some(error)) } + /** Check if the receiver has started or not. */ def isStarted(): Boolean = { executor.isReceiverStarted() } - /** Check if receiver has been marked for stopping. */ + /** + * Check if receiver has been marked for stopping. Use this to identify when + * the receiving of data should be stopped. + */ def isStopped(): Boolean = { executor.isReceiverStopped() } - /** Get unique identifier of this receiver. */ + /** + * Get the unique identifier the receiver input stream that this + * receiver is associated with. + */ def streamId = id /* diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java new file mode 100644 index 0000000000000..1b0787fe69dec --- /dev/null +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import static org.junit.Assert.*; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.receiver.Receiver; +import org.apache.spark.api.java.function.Function; + +import java.io.BufferedReader; +import java.io.InputStreamReader; +import java.io.Serializable; +import java.net.ConnectException; +import java.net.Socket; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +public class JavaReceiverAPISuite implements Serializable { + + @Before + public void setUp() { + System.clearProperty("spark.streaming.clock"); + } + + @After + public void tearDown() { + System.clearProperty("spark.streaming.clock"); + } + + @Test + public void testReceiver() throws InterruptedException { + TestServer server = new TestServer(0); + server.start(); + + final AtomicLong dataCounter = new AtomicLong(0); + + try { + JavaStreamingContext ssc = new JavaStreamingContext("local[2]", "test", new Duration(200)); + JavaReceiverInputDStream input = + ssc.receiverStream(new JavaSocketReceiver("localhost", server.port())); + JavaDStream mapped = input.map(new Function() { + @Override + public String call(String v1) throws Exception { + return v1 + "."; + } + }); + mapped.foreachRDD(new Function, Void>() { + @Override + public Void call(JavaRDD rdd) throws Exception { + long count = rdd.count(); + dataCounter.addAndGet(count); + return null; + } + }); + + ssc.start(); + long startTime = System.currentTimeMillis(); + long timeout = 10000; + + Thread.sleep(200); + for (int i = 0; i < 6; i++) { + server.send("" + i + "\n"); // \n to make sure these are separate lines + Thread.sleep(100); + } + while (dataCounter.get() == 0 && System.currentTimeMillis() - startTime < timeout) { + Thread.sleep(100); + } + ssc.stop(); + assertTrue(dataCounter.get() > 0); + } finally { + server.stop(); + } + } +} + +class JavaSocketReceiver extends Receiver { + + String host = null; + int port = -1; + + public JavaSocketReceiver(String host_ , int port_) { + super(StorageLevel.MEMORY_AND_DISK()); + host = host_; + port = port_; + } + + @Override + public void onStart() { + new Thread() { + @Override public void run() { + receive(); + } + }.start(); + } + + @Override + public void onStop() { + } + + private void receive() { + Socket socket = null; + try { + socket = new Socket(host, port); + BufferedReader in = new BufferedReader(new InputStreamReader(socket.getInputStream())); + String userInput; + while ((userInput = in.readLine()) != null) { + store(userInput); + } + in.close(); + socket.close(); + } catch(ConnectException ce) { + ce.printStackTrace(); + restart("Could not connect", ce); + } catch(Throwable t) { + t.printStackTrace(); + restart("Error receiving data", t); + } + } +} + diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala index 33f6df8f88177..c0ea0491c313d 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala @@ -26,6 +26,7 @@ import org.apache.spark.streaming._ import java.util.ArrayList import collection.JavaConversions._ import org.apache.spark.api.java.JavaRDDLike +import org.apache.spark.streaming.dstream.DStream /** Exposes streaming test functionality in a Java-friendly way. */ trait JavaTestBase extends TestSuiteBase { @@ -51,8 +52,7 @@ trait JavaTestBase extends TestSuiteBase { * [[org.apache.spark.streaming.TestOutputStream]]. **/ def attachTestOutputStream[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]]( - dstream: JavaDStreamLike[T, This, R]) = - { + dstream: JavaDStreamLike[T, This, R]) = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val ostream = new TestOutputStreamWithPartitions(dstream.dstream) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index b55b7834c90c1..3fa254065cc44 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -49,7 +49,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) - val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) + val networkStream = ssc.socketTextStream( + "localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] val outputStream = new TestOutputStream(networkStream, outputBuffer) def output = outputBuffer.flatMap(x => x) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 3e2b25af84098..ee0bc8b7d6a71 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -165,7 +165,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc = new StreamingContext(sc, Milliseconds(100)) var runningCount = 0 TestReceiver.counter.set(1) - val input = ssc.networkStream(new TestReceiver) + val input = ssc.receiverStream(new TestReceiver) input.count.foreachRDD(rdd => { val count = rdd.first() runningCount += count.toInt diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 4f63fd37822cb..8036f77c973ae 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -155,6 +155,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { def afterFunction() { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") + System.clearProperty("spark.streaming.clock") } before(beforeFunction) From 8421034e793c0960373a0a1d694ce334ad36e747 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 28 Apr 2014 13:58:42 -0700 Subject: [PATCH 199/641] Changes to dev release script --- dev/create-release/create-release.sh | 59 +++++++++++++++------------- 1 file changed, 32 insertions(+), 27 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index f1aa0f8f99e13..ae1b41544443c 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -21,47 +21,52 @@ # Publishes releases to Maven and packages/copies binary release artifacts. # Expects to be run in a totally empty directory. # +# Options: +# --package-only only packages an existing release candidate +# # Would be nice to add: # - Send output to stderr and have useful logging in stdout -# - Have this use sbt rather than Maven release plug in -GIT_USERNAME=pwendell -GIT_PASSWORD=XXX -GPG_PASSPHRASE=XXX -GIT_BRANCH=branch-0.9 -RELEASE_VERSION=0.9.0-incubating -RC_NAME=rc2 -USER_NAME=pwendell +GIT_USERNAME=${GIT_USERNAME:-pwendell} +GIT_PASSWORD=${GIT_PASSWORD:-XXX} +GPG_PASSPHRASE=${GPG_PASSPHRASE:-XXX} +GIT_BRANCH=${GIT_BRANCH:-branch-1.0} +RELEASE_VERSION=${RELEASE_VERSION:-1.0.0} +RC_NAME=${RC_NAME:-rc2} +USER_NAME=${USER_NAME:-pwendell} set -e GIT_TAG=v$RELEASE_VERSION-$RC_NAME -# Artifact publishing - -git clone https://git-wip-us.apache.org/repos/asf/spark.git -b $GIT_BRANCH -cd spark -export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" +if [[ ! "$@" =~ --package-only ]]; then + echo "Creating and publishing release" + # Artifact publishing + git clone https://git-wip-us.apache.org/repos/asf/spark.git -b $GIT_BRANCH + cd spark + export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" -mvn -Pyarn release:clean + mvn -Pyarn release:clean -mvn -DskipTests \ - -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ - -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ - -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Pspark-ganglia-lgpl\ - -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ - --batch-mode release:prepare + mvn -DskipTests \ + -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ + -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ + -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ + -Pyarn -Phive -Pspark-ganglia-lgpl\ + -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ + --batch-mode release:prepare -mvn -DskipTests \ - -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ - -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Pspark-ganglia-lgpl\ - release:perform + mvn -DskipTests \ + -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ + -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ + -Pyarn -Phive -Pspark-ganglia-lgpl\ + release:perform -rm -rf spark + rm -rf spark +fi # Source and binary tarballs +echo "Packaging release tarballs" git clone https://git-wip-us.apache.org/repos/asf/spark.git cd spark git checkout --force $GIT_TAG From cae054aaf41ca0ee585231896db67169b61af689 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 28 Apr 2014 17:26:57 -0700 Subject: [PATCH 200/641] SPARK-1652: Spark submit should fail gracefully if YARN not enabled Author: Patrick Wendell Closes #579 from pwendell/spark-submit-yarn-2 and squashes the following commits: 05e1b11 [Patrick Wendell] Small fix d2a40ad [Patrick Wendell] SPARK-1652: Spark submit should fail gracefully if YARN support not enabled --- .../scala/org/apache/spark/deploy/SparkSubmit.scala | 10 ++++++++++ core/src/main/scala/org/apache/spark/util/Utils.scala | 6 ++++++ 2 files changed, 16 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 24edc60684376..c463ee09993a2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -23,6 +23,7 @@ import java.net.{URI, URL} import scala.collection.mutable.{ArrayBuffer, HashMap, Map} import org.apache.spark.executor.ExecutorURLClassLoader +import org.apache.spark.util.Utils /** * Scala code behind the spark-submit script. The script handles setting up the classpath with @@ -128,6 +129,15 @@ object SparkSubmit { childArgs += ("--class", appArgs.mainClass) } + if (clusterManager == YARN) { + // The choice of class is arbitrary, could use any spark-yarn class + if (!Utils.classIsLoadable("org.apache.spark.deploy.yarn.Client") && !Utils.isTesting) { + val msg = "Could not load YARN classes. This copy of Spark may not have been compiled " + + "with YARN support." + throw new Exception(msg) + } + } + val options = List[OptionAssigner]( new OptionAssigner(appArgs.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), new OptionAssigner(appArgs.driverExtraClassPath, STANDALONE | YARN, true, diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index b678604ff81c8..79f314c8dd36c 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -28,6 +28,7 @@ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag +import scala.util.Try import com.google.common.io.Files import org.apache.commons.lang.SystemUtils @@ -137,6 +138,11 @@ private[spark] object Utils extends Logging { def getContextOrSparkClassLoader = Option(Thread.currentThread().getContextClassLoader).getOrElse(getSparkClassLoader) + /** Determines whether the provided class is loadable in the current thread. */ + def classIsLoadable(clazz: String): Boolean = { + Try { Class.forName(clazz, false, getContextOrSparkClassLoader) }.isSuccess + } + /** * Primitive often used when writing {@link java.nio.ByteBuffer} to {@link java.io.DataOutput}. */ From 949e393101e19cd00591a9930c4b364278e22609 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 28 Apr 2014 17:29:22 -0700 Subject: [PATCH 201/641] SPARK-1654 and SPARK-1653: Fixes in spark-submit. Deals with two issues: 1. Spark shell didn't correctly pass quoted arguments to spark-submit. ```./bin/spark-shell --driver-java-options "-Dfoo=f -Dbar=b"``` 2. Spark submit used deprecated environment variables (SPARK_CLASSPATH) which triggered warnings. Now we use new, more narrowly scoped, variables. Author: Patrick Wendell Closes #576 from pwendell/spark-submit and squashes the following commits: 67004c9 [Patrick Wendell] SPARK-1654 and SPARK-1653: Fixes in spark-submit. --- bin/compute-classpath.sh | 2 +- bin/spark-class | 9 +++++---- bin/spark-shell | 11 +++++------ bin/spark-submit | 6 +++--- .../apache/spark/deploy/SparkSubmitArguments.scala | 4 +++- 5 files changed, 17 insertions(+), 15 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 3a59f599fd7d2..b0218531e9eb8 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -28,7 +28,7 @@ FWDIR="$(cd `dirname $0`/..; pwd)" . $FWDIR/bin/load-spark-env.sh # Build up classpath -CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" +CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH:$FWDIR/conf" ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION" diff --git a/bin/spark-class b/bin/spark-class index 6871e180c9fa8..e8160c8af64c1 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -73,11 +73,13 @@ case "$1" in OUR_JAVA_MEM=${SPARK_EXECUTOR_MEMORY:-$DEFAULT_MEM} ;; - # All drivers use SPARK_JAVA_OPTS + SPARK_DRIVER_MEMORY. The repl also uses SPARK_REPL_OPTS. - 'org.apache.spark.repl.Main') - OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_REPL_OPTS" + # Spark submit uses SPARK_SUBMIT_OPTS and SPARK_JAVA_OPTS + 'org.apache.spark.deploy.SparkSubmit') + OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_SUBMIT_OPTS \ + -Djava.library.path=$SPARK_SUBMIT_LIBRARY_PATH" OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} ;; + *) OUR_JAVA_OPTS="$SPARK_JAVA_OPTS" OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} @@ -98,7 +100,6 @@ fi # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="$OUR_JAVA_OPTS" -JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$_SPARK_LIBRARY_PATH" JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e "$FWDIR/conf/java-opts" ] ; then diff --git a/bin/spark-shell b/bin/spark-shell index f1f3c18877ed4..7f03349c5e910 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -20,7 +20,6 @@ # # Shell script for starting the Spark Shell REPL -args="$@" cygwin=false case "`uname`" in CYGWIN*) cygwin=true;; @@ -46,12 +45,12 @@ function main(){ # "Backspace sends ^H" setting in "Keys" section of the Mintty options # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 - export SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-submit spark-internal "$args" --class org.apache.spark.repl.Main + export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" + $FWDIR/bin/spark-submit spark-internal "$@" --class org.apache.spark.repl.Main stty icanon echo > /dev/null 2>&1 else - export SPARK_REPL_OPTS - $FWDIR/bin/spark-submit spark-internal "$args" --class org.apache.spark.repl.Main + export SPARK_SUBMIT_OPTS + $FWDIR/bin/spark-submit spark-internal "$@" --class org.apache.spark.repl.Main fi } @@ -83,7 +82,7 @@ if [[ ! $? ]]; then saved_stty="" fi -main +main "$@" # record the exit status lest it be overwritten: # then reenable echo and propagate the code. diff --git a/bin/spark-submit b/bin/spark-submit index b2a1dca721dff..dd0d95d9d4002 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -26,11 +26,11 @@ while (($#)); do elif [ "$1" = "--driver-memory" ]; then DRIVER_MEMORY=$2 elif [ "$1" = "--driver-library-path" ]; then - export _SPARK_LIBRARY_PATH=$2 + export SPARK_SUBMIT_LIBRARY_PATH=$2 elif [ "$1" = "--driver-class-path" ]; then - export SPARK_CLASSPATH="$SPARK_CLASSPATH:$2" + export SPARK_SUBMIT_CLASSPATH=$2 elif [ "$1" = "--driver-java-options" ]; then - export SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $2" + export SPARK_SUBMIT_OPTS=$2 fi shift done diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 58d9e9add764a..5834dc40f1f95 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -298,7 +298,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). | --driver-java-options Extra Java options to pass to the driver | --driver-library-path Extra library path entries to pass to the driver - | --driver-class-path Extra class path entries to pass to the driver + | --driver-class-path Extra class path entries to pass to the driver. Note that + | jars added with --jars are automatically included in the + | classpath. | | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G). | From 9f7a095184d6c7a9b1bbac55efcc3d878f876768 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 28 Apr 2014 18:14:59 -0700 Subject: [PATCH 202/641] SPARK-1652: Remove incorrect deprecation warning in spark-submit This is a straightforward fix. Author: Patrick Wendell This patch had conflicts when merged, resolved by Committer: Patrick Wendell Closes #578 from pwendell/spark-submit-yarn and squashes the following commits: 96027c7 [Patrick Wendell] Test fixes b5be173 [Patrick Wendell] Review feedback 4ac9cac [Patrick Wendell] SPARK-1652: spark-submit for yarn prints warnings even though calling as expected --- .../main/scala/org/apache/spark/deploy/SparkSubmit.scala | 3 +++ .../scala/org/apache/spark/deploy/SparkSubmitSuite.scala | 5 +++-- .../main/scala/org/apache/spark/deploy/yarn/Client.scala | 6 ++++-- .../main/scala/org/apache/spark/deploy/yarn/Client.scala | 6 ++++-- 4 files changed, 14 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index c463ee09993a2..d131f1809c99e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -137,6 +137,9 @@ object SparkSubmit { throw new Exception(msg) } } + + // Special flag to avoid deprecation warnings at the client + sysProps("SPARK_SUBMIT") = "true" val options = List[OptionAssigner]( new OptionAssigner(appArgs.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 10a65c75cc621..b3541b4a40b79 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -122,7 +122,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { childArgsStr should include ("--num-executors 6") mainClass should be ("org.apache.spark.deploy.yarn.Client") classpath should have length (0) - sysProps should have size (0) + sysProps should have size (1) } test("handles YARN client mode") { @@ -146,6 +146,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { sysProps("spark.yarn.dist.files") should be ("file1.txt,file2.txt") sysProps("spark.yarn.dist.archives") should be ("archive1.txt,archive2.txt") sysProps("spark.executor.instances") should be ("6") + sysProps("SPARK_SUBMIT") should be ("true") } test("handles standalone cluster mode") { @@ -159,7 +160,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { childArgsStr should include ("launch spark://h:p thejar.jar org.SomeClass arg1 arg2") mainClass should be ("org.apache.spark.deploy.Client") classpath should have length (0) - sysProps should have size (1) // contains --jar entry + sysProps should have size (2) // contains --jar entry and SPARK_SUBMIT } test("handles standalone client mode") { diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 00c7649e68e13..8226207de42b8 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -169,8 +169,10 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa object Client { def main(argStrings: Array[String]) { - println("WARNING: This client is deprecated and will be removed in a future version of Spark.") - println("Use ./bin/spark-submit with \"--master yarn\"") + if (!sys.props.contains("SPARK_SUBMIT")) { + println("WARNING: This client is deprecated and will be removed in a " + + "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"") + } // Set an env variable indicating we are running in YARN mode. // Note that anything with SPARK prefix gets propagated to all (remote) processes diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 2e2fb5d4fa787..1b6bfb42a5c1c 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -171,8 +171,10 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa object Client { def main(argStrings: Array[String]) { - println("WARNING: This client is deprecated and will be removed in a future version of Spark.") - println("Use ./bin/spark-submit with \"--master yarn\"") + if (!sys.props.contains("SPARK_SUBMIT")) { + println("WARNING: This client is deprecated and will be removed in a " + + "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"") + } // Set an env variable indicating we are running in YARN mode. // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes - From 030f2c2126d5075576cd6d83a1ee7462c48b953b Mon Sep 17 00:00:00 2001 From: witgo Date: Mon, 28 Apr 2014 22:50:51 -0700 Subject: [PATCH 203/641] Improved build configuration 1, Fix SPARK-1441: compile spark core error with hadoop 0.23.x 2, Fix SPARK-1491: maven hadoop-provided profile fails to build 3, Fix org.scala-lang: * ,org.apache.avro:* inconsistent versions dependency 4, A modified on the sql/catalyst/pom.xml,sql/hive/pom.xml,sql/core/pom.xml (Four spaces formatted into two spaces) Author: witgo Closes #480 from witgo/format_pom and squashes the following commits: 03f652f [witgo] review commit b452680 [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom bee920d [witgo] revert fix SPARK-1629: Spark Core missing commons-lang dependence 7382a07 [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom 6902c91 [witgo] fix SPARK-1629: Spark Core missing commons-lang dependence 0da4bc3 [witgo] merge master d1718ed [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom e345919 [witgo] add avro dependency to yarn-alpha 77fad08 [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom 62d0862 [witgo] Fix org.scala-lang: * inconsistent versions dependency 1a162d7 [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom 934f24d [witgo] review commit cf46edc [witgo] exclude jruby 06e7328 [witgo] Merge branch 'SparkBuild' into format_pom 99464d2 [witgo] fix maven hadoop-provided profile fails to build 0c6c1fc [witgo] Fix compile spark core error with hadoop 0.23.x 6851bec [witgo] Maintain consistent SparkBuild.scala, pom.xml --- bagel/pom.xml | 14 --- core/pom.xml | 22 ----- docs/building-with-maven.md | 10 +- examples/pom.xml | 18 +--- external/flume/pom.xml | 14 --- external/kafka/pom.xml | 14 --- external/mqtt/pom.xml | 14 --- external/twitter/pom.xml | 14 --- external/zeromq/pom.xml | 14 --- graphx/pom.xml | 14 --- make-distribution.sh | 15 ++- mllib/pom.xml | 14 --- pom.xml | 79 +++++++++++++-- project/SparkBuild.scala | 9 +- repl/pom.xml | 14 --- sql/catalyst/pom.xml | 109 ++++++++++----------- sql/core/pom.xml | 122 +++++++++++------------ sql/hive/pom.xml | 186 +++++++++++++++++------------------- streaming/pom.xml | 14 --- tools/pom.xml | 14 --- yarn/alpha/pom.xml | 14 --- yarn/pom.xml | 9 -- yarn/stable/pom.xml | 14 --- 23 files changed, 295 insertions(+), 466 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 142f75c5d2c64..355f437c5b16a 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -31,20 +31,6 @@ Spark Project Bagel http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/core/pom.xml b/core/pom.xml index 058b7acba73ca..73f573a414050 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -30,19 +30,6 @@ jar Spark Project Core http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.hadoop @@ -147,15 +134,6 @@ org.json4s json4s-jackson_${scala.binary.version} 3.2.6 - - - - org.scala-lang - scalap - - colt diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 771d1933a13ea..a5e5303467401 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -39,7 +39,10 @@ For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions wit # Cloudera CDH 4.2.0 with MapReduce v1 $ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -DskipTests clean package -For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you should enable the "yarn-alpha" or "yarn" profile and set the "hadoop.version", "yarn.version" property: + # Apache Hadoop 0.23.x + $ mvn -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package + +For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you can enable the "yarn-alpha" or "yarn" profile and set the "hadoop.version", "yarn.version" property. Note that Hadoop 0.23.X requires a special `-Phadoop-0.23` profile: # Apache Hadoop 2.0.5-alpha $ mvn -Pyarn-alpha -Dhadoop.version=2.0.5-alpha -Dyarn.version=2.0.5-alpha -DskipTests clean package @@ -47,9 +50,12 @@ For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with # Cloudera CDH 4.2.0 with MapReduce v2 $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-cdh4.2.0 -DskipTests clean package - # Apache Hadoop 2.2.X ( e.g. 2.2.0 as below ) and newer + # Apache Hadoop 2.2.X (e.g. 2.2.0 as below) and newer $ mvn -Pyarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -DskipTests clean package + # Apache Hadoop 0.23.x + $ mvn -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.7 -Dyarn.version=0.23.7 -DskipTests clean package + ## Spark Tests in Maven ## Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). Some of the require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. You can then run the tests with `mvn -Dhadoop.version=... test`. diff --git a/examples/pom.xml b/examples/pom.xml index 704d6df7c57f9..a2d1b19736615 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -31,20 +31,6 @@ Spark Project Examples http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark @@ -124,6 +110,10 @@ commons-logging commons-logging + + org.jruby + jruby-complete + diff --git a/external/flume/pom.xml b/external/flume/pom.xml index b84ca0c5e1dd0..03d3b2394f510 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -31,20 +31,6 @@ Spark Project External Flume http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 343e1fabd823f..979eb0ca624bd 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -31,20 +31,6 @@ Spark Project External Kafka http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 3710a63541d78..9aa1c1a9f5b80 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -31,20 +31,6 @@ Spark Project External MQTT http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 398b9f4fbaa7d..a443459594710 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -31,20 +31,6 @@ Spark Project External Twitter http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 77e957f404645..a40e55876e640 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -31,20 +31,6 @@ Spark Project External ZeroMQ http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/graphx/pom.xml b/graphx/pom.xml index b4c67ddcd8ca9..dc108d2fe7fbd 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -31,20 +31,6 @@ Spark Project GraphX http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/make-distribution.sh b/make-distribution.sh index 4ac80efae0ab2..c05dcd89d90a7 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -123,10 +123,19 @@ else fi if [ "$SPARK_YARN" == "true" ]; then - mvn clean package -DskipTests -Pyarn -Dhadoop.version=$SPARK_HADOOP_VERSION \ - -Dyarn.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE + if [[ "$SPARK_HADOOP_VERSION" =~ "0.23." ]]; then + mvn clean package -DskipTests -Pyarn-alpha -Dhadoop.version=$SPARK_HADOOP_VERSION \ + -Dyarn.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE -Phadoop-0.23 + else + mvn clean package -DskipTests -Pyarn -Dhadoop.version=$SPARK_HADOOP_VERSION \ + -Dyarn.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE + fi else - mvn clean package -DskipTests -Dhadoop.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE + if [[ "$SPARK_HADOOP_VERSION" =~ "0.23." ]]; then + mvn clean package -Phadoop-0.23 -DskipTests -Dhadoop.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE + else + mvn clean package -DskipTests -Dhadoop.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE + fi fi # Make directories diff --git a/mllib/pom.xml b/mllib/pom.xml index e7ce00efc4af6..cdd33dbb7970d 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -31,20 +31,6 @@ Spark Project ML Library http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/pom.xml b/pom.xml index e9117998252fc..646753fe30301 100644 --- a/pom.xml +++ b/pom.xml @@ -127,6 +127,7 @@ 8.1.14.v20131031 0.3.6 3.0.0 + 1.7.4 64m 512m @@ -293,7 +294,7 @@ org.ow2.asm asm-commons - + @@ -308,7 +309,7 @@ org.ow2.asm asm-commons - + @@ -425,6 +426,16 @@ scala-library ${scala.version} + + org.scala-lang + scala-actors + ${scala.version} + + + org.scala-lang + scalap + ${scala.version} + org.scalatest scalatest_${scala.binary.version} @@ -492,7 +503,45 @@ org.apache.avro avro - 1.7.4 + ${avro.version} + + + org.jboss.netty + netty + + + io.netty + netty + + + + + org.apache.avro + avro-ipc + ${avro.version} + + + org.mortbay.jetty + jetty + + + org.mortbay.jetty + jetty-util + + + org.mortbay.jetty + servlet-api + + + org.apache.velocity + velocity + + + + + org.apache.avro + avro-mapred + ${avro.version} org.jboss.netty @@ -687,7 +736,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - + @@ -742,7 +791,7 @@ build-helper-maven-plugin - add-scala-sources + add-scala-sources generate-sources add-source @@ -779,6 +828,17 @@
      + + + hadoop-0.23 + + + org.apache.avro + avro + + + + yarn-alpha @@ -790,6 +850,12 @@ yarn + + + org.apache.avro + avro + + @@ -838,7 +904,7 @@ - + hadoop-provided @@ -878,6 +944,7 @@ org.apache.zookeeper zookeeper + 3.4.5 provided diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5adfbe373cb00..a9504f3926082 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -106,7 +106,7 @@ object SparkBuild extends Build { case Some(v) => v.toBoolean } 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() + val maybeAvro = if (hadoopVersion.startsWith("0.23.")) Seq("org.apache.avro" % "avro" % "1.7.4") else Seq() lazy val isHiveEnabled = Properties.envOrNone("SPARK_HIVE") match { case None => DEFAULT_HIVE @@ -224,7 +224,7 @@ object SparkBuild extends Build { org.apache apache - 13 + 14 http://spark.apache.org/ @@ -250,7 +250,7 @@ object SparkBuild extends Build { JIRA - https://spark-project.atlassian.net/browse/SPARK + https://issues.apache.org/jira/browse/SPARK ), @@ -313,6 +313,7 @@ object SparkBuild extends Build { val excludeCurator = ExclusionRule(organization = "org.apache.curator") val excludePowermock = ExclusionRule(organization = "org.powermock") val excludeFastutil = ExclusionRule(organization = "it.unimi.dsi") + val excludeJruby = ExclusionRule(organization = "org.jruby") val excludeThrift = ExclusionRule(organization = "org.apache.thrift") def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark", @@ -417,7 +418,7 @@ object SparkBuild extends Build { v => "spark-examples-" + v + "-hadoop" + hadoopVersion + ".jar" }, libraryDependencies ++= Seq( "com.twitter" %% "algebird-core" % "0.1.11", - "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging), + "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging, excludeJruby), "org.apache.cassandra" % "cassandra-all" % "1.2.6" exclude("com.google.guava", "guava") exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru") diff --git a/repl/pom.xml b/repl/pom.xml index 78d2fe13c27eb..b761a176ce256 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -31,20 +31,6 @@ Spark Project REPL http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - /usr/share/spark root diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 9d5c6a857bb00..8d2e4baf69e30 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -16,67 +16,56 @@ ~ limitations under the License. --> - - 4.0.0 - - org.apache.spark - spark-parent - 1.0.0-SNAPSHOT - ../../pom.xml - - + + 4.0.0 + org.apache.spark - spark-catalyst_2.10 - jar - Spark Project Catalyst - http://spark.apache.org/ + spark-parent + 1.0.0-SNAPSHOT + ../../pom.xml + - - - yarn-alpha - - - org.apache.avro - avro - - - - + org.apache.spark + spark-catalyst_2.10 + jar + Spark Project Catalyst + http://spark.apache.org/ - - - org.scala-lang - scala-reflect - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - com.typesafe - scalalogging-slf4j_${scala.binary.version} - 1.0.1 - - - org.scalatest - scalatest_${scala.binary.version} - test - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - - + + + org.scala-lang + scala-reflect + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + com.typesafe + scalalogging-slf4j_${scala.binary.version} + 1.0.1 + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 85580ed6b822f..fb3b190b4ec5a 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -16,72 +16,62 @@ ~ limitations under the License. --> - - 4.0.0 - - org.apache.spark - spark-parent - 1.0.0-SNAPSHOT - ../../pom.xml - - + + 4.0.0 + org.apache.spark - spark-sql_2.10 - jar - Spark Project SQL - http://spark.apache.org/ - - - yarn-alpha - - - org.apache.avro - avro - - - - + spark-parent + 1.0.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-sql_2.10 + jar + Spark Project SQL + http://spark.apache.org/ - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-catalyst_${scala.binary.version} - ${project.version} - - - com.twitter - parquet-column - ${parquet.version} - - - com.twitter - parquet-hadoop - ${parquet.version} - - - org.scalatest - scalatest_${scala.binary.version} - test - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - - + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${project.version} + + + com.twitter + parquet-column + ${parquet.version} + + + com.twitter + parquet-hadoop + ${parquet.version} + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index a662da76ce25a..889d249146b8c 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -16,105 +16,95 @@ ~ limitations under the License. --> - - 4.0.0 - - org.apache.spark - spark-parent - 1.0.0-SNAPSHOT - ../../pom.xml - - + + 4.0.0 + org.apache.spark - spark-hive_2.10 - jar - Spark Project Hive - http://spark.apache.org/ - - - yarn-alpha - - - org.apache.avro - avro - - - - + spark-parent + 1.0.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-hive_2.10 + jar + Spark Project Hive + http://spark.apache.org/ - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-sql_${scala.binary.version} - ${project.version} - - - org.apache.hive - hive-metastore - ${hive.version} - - - org.apache.hive - hive-exec - ${hive.version} - - - org.codehaus.jackson - jackson-mapper-asl - - - org.apache.hive - hive-serde - ${hive.version} - - - org.scalatest - scalatest_${scala.binary.version} - test - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + + + org.apache.hive + hive-metastore + ${hive.version} + + + org.apache.hive + hive-exec + ${hive.version} + + + org.codehaus.jackson + jackson-mapper-asl + + + org.apache.hive + hive-serde + ${hive.version} + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + - - - org.apache.maven.plugins - maven-dependency-plugin - 2.4 - - - copy-dependencies - package - - copy-dependencies - - - - ${basedir}/../../lib_managed/jars - false - false - true - org.datanucleus - - - - - - + + + org.apache.maven.plugins + maven-dependency-plugin + 2.4 + + + copy-dependencies + package + + copy-dependencies + + + + ${basedir}/../../lib_managed/jars + false + false + true + org.datanucleus + + + + + + diff --git a/streaming/pom.xml b/streaming/pom.xml index 93b1c5a37aff9..6435224a14674 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -31,20 +31,6 @@ Spark Project Streaming http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/tools/pom.xml b/tools/pom.xml index ae2ba64e07c21..1875c497bc61c 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -30,20 +30,6 @@ Spark Project Tools http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index d0aeaceb0d23c..e076ca1d44b97 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -24,20 +24,6 @@ ../pom.xml - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark spark-yarn-alpha_2.10 jar diff --git a/yarn/pom.xml b/yarn/pom.xml index 3342cb65edcd1..02f36627431b9 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -70,15 +70,6 @@ alpha - - - - - org.apache.avro - avro - - diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index e7915d12aef63..0780f251b595c 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -24,20 +24,6 @@ ../pom.xml - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark spark-yarn_2.10 jar From 719c8bc65e8a4096d09a6a9f15c90da3ddd7d9d4 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 29 Apr 2014 00:10:17 -0700 Subject: [PATCH 204/641] HOTFIX: Bug in release script --- dev/create-release/create-release.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index ae1b41544443c..44da8f7cbf53f 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -62,6 +62,7 @@ if [[ ! "$@" =~ --package-only ]]; then -Pyarn -Phive -Pspark-ganglia-lgpl\ release:perform + cd .. rm -rf spark fi From 497be3ca2d8f0600e927f8f036177fcd3bb6e229 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 29 Apr 2014 00:36:15 -0700 Subject: [PATCH 205/641] Minor fix to python table caching API. Author: Michael Armbrust Closes #585 from marmbrus/pythonCacheTable and squashes the following commits: 7ec1f91 [Michael Armbrust] Minor fix to python table caching API. --- python/pyspark/sql.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 114fa138d0de2..1a62031db5c41 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -143,13 +143,13 @@ def table(self, tableName): """ return SchemaRDD(self._ssql_ctx.table(tableName), self) - def cacheTable(tableName): + def cacheTable(self, tableName): """ Caches the specified table in-memory. """ self._ssql_ctx.cacheTable(tableName) - def uncacheTable(tableName): + def uncacheTable(self, tableName): """ Removes the specified table from the in-memory cache. """ From 3f38334f441940ed0a5bbf5588ca7f22d3940359 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 29 Apr 2014 00:41:03 -0700 Subject: [PATCH 206/641] [SPARK-1636][MLLIB] Move main methods to examples * `NaiveBayes` -> `SparseNaiveBayes` * `KMeans` -> `DenseKMeans` * `SVMWithSGD` and `LogisticRegerssionWithSGD` -> `BinaryClassification` * `ALS` -> `MovieLensALS` * `LinearRegressionWithSGD`, `LassoWithSGD`, and `RidgeRegressionWithSGD` -> `LinearRegression` * `DecisionTree` -> `DecisionTreeRunner` `scopt` is used for parsing command-line parameters. `scopt` has MIT license and it only depends on `scala-library`. Example help message: ~~~ BinaryClassification: an example app for binary classification. Usage: BinaryClassification [options] --numIterations number of iterations --stepSize initial step size, default: 1.0 --algorithm algorithm (SVM,LR), default: LR --regType regularization type (L1,L2), default: L2 --regParam regularization parameter, default: 0.1 input paths to labeled examples in LIBSVM format ~~~ Author: Xiangrui Meng Closes #584 from mengxr/mllib-main and squashes the following commits: 7b58c60 [Xiangrui Meng] minor 6e35d7e [Xiangrui Meng] make imports explicit and fix code style c6178c9 [Xiangrui Meng] update TS PCA/SVD to use new spark-submit 6acff75 [Xiangrui Meng] use scopt for DecisionTreeRunner be86069 [Xiangrui Meng] use main instead of extending App b3edf68 [Xiangrui Meng] move DecisionTree's main method to examples 8bfaa5a [Xiangrui Meng] change NaiveBayesParams to Params fe23dcb [Xiangrui Meng] remove main from KMeans and add DenseKMeans as an example 67f4448 [Xiangrui Meng] remove main methods from linear regression algorithms and add LinearRegression example b066bbc [Xiangrui Meng] remove main from ALS and add MovieLensALS example b040f3b [Xiangrui Meng] change BinaryClassificationParams to Params 577945b [Xiangrui Meng] remove unused imports from NB 3d299bc [Xiangrui Meng] remove main from LR/SVM and add an example app for binary classification f70878e [Xiangrui Meng] remove main from NaiveBayes and add an example NaiveBayes app 01ec2cd [Xiangrui Meng] Merge branch 'master' into mllib-main 9420692 [Xiangrui Meng] add scopt to examples dependencies --- examples/pom.xml | 5 + .../examples/mllib/BinaryClassification.scala | 145 ++++++++++++++++ .../examples/mllib/DecisionTreeRunner.scala | 161 ++++++++++++++++++ .../spark/examples/mllib/DenseKMeans.scala | 109 ++++++++++++ .../examples/mllib/LinearRegression.scala | 125 ++++++++++++++ .../spark/examples/mllib/MovieLensALS.scala | 131 ++++++++++++++ .../examples/mllib/SparseNaiveBayes.scala | 102 +++++++++++ .../spark/examples/mllib/TallSkinnyPCA.scala | 12 +- .../spark/examples/mllib/TallSkinnySVD.scala | 12 +- .../classification/LogisticRegression.scala | 18 +- .../mllib/classification/NaiveBayes.scala | 22 +-- .../spark/mllib/classification/SVM.scala | 18 +- .../spark/mllib/clustering/KMeans.scala | 25 +-- .../spark/mllib/recommendation/ALS.scala | 45 +---- .../apache/spark/mllib/regression/Lasso.scala | 17 -- .../mllib/regression/LinearRegression.scala | 16 -- .../mllib/regression/RidgeRegression.scala | 19 --- .../spark/mllib/tree/DecisionTree.scala | 131 +------------- project/SparkBuild.scala | 3 +- 19 files changed, 795 insertions(+), 321 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala diff --git a/examples/pom.xml b/examples/pom.xml index a2d1b19736615..e1fc149d87f17 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -166,6 +166,11 @@ + + com.github.scopt + scopt_${scala.binary.version} + 3.2.0 + diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala new file mode 100644 index 0000000000000..ec9de022c1d47 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala @@ -0,0 +1,145 @@ +/* + * 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.examples.mllib + +import org.apache.log4j.{Level, Logger} +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.classification.{LogisticRegressionWithSGD, SVMWithSGD} +import org.apache.spark.mllib.evaluation.binary.BinaryClassificationMetrics +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.optimization.{SquaredL2Updater, L1Updater} + +/** + * An example app for binary classification. Run with + * {{{ + * ./bin/run-example org.apache.spark.examples.mllib.BinaryClassification + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object BinaryClassification { + + object Algorithm extends Enumeration { + type Algorithm = Value + val SVM, LR = Value + } + + object RegType extends Enumeration { + type RegType = Value + val L1, L2 = Value + } + + import Algorithm._ + import RegType._ + + case class Params( + input: String = null, + numIterations: Int = 100, + stepSize: Double = 1.0, + algorithm: Algorithm = LR, + regType: RegType = L2, + regParam: Double = 0.1) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("BinaryClassification") { + head("BinaryClassification: an example app for binary classification.") + opt[Int]("numIterations") + .text("number of iterations") + .action((x, c) => c.copy(numIterations = x)) + opt[Double]("stepSize") + .text(s"initial step size, default: ${defaultParams.stepSize}") + .action((x, c) => c.copy(stepSize = x)) + opt[String]("algorithm") + .text(s"algorithm (${Algorithm.values.mkString(",")}), " + + s"default: ${defaultParams.algorithm}") + .action((x, c) => c.copy(algorithm = Algorithm.withName(x))) + opt[String]("regType") + .text(s"regularization type (${RegType.values.mkString(",")}), " + + s"default: ${defaultParams.regType}") + .action((x, c) => c.copy(regType = RegType.withName(x))) + opt[Double]("regParam") + .text(s"regularization parameter, default: ${defaultParams.regParam}") + arg[String]("") + .required() + .text("input paths to labeled examples in LIBSVM format") + .action((x, c) => c.copy(input = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"BinaryClassification with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val examples = MLUtils.loadLibSVMData(sc, params.input).cache() + + val splits = examples.randomSplit(Array(0.8, 0.2)) + val training = splits(0).cache() + val test = splits(1).cache() + + val numTraining = training.count() + val numTest = test.count() + println(s"Training: $numTraining, test: $numTest.") + + examples.unpersist(blocking = false) + + val updater = params.regType match { + case L1 => new L1Updater() + case L2 => new SquaredL2Updater() + } + + val model = params.algorithm match { + case LR => + val algorithm = new LogisticRegressionWithSGD() + algorithm.optimizer + .setNumIterations(params.numIterations) + .setStepSize(params.stepSize) + .setUpdater(updater) + .setRegParam(params.regParam) + algorithm.run(training).clearThreshold() + case SVM => + val algorithm = new SVMWithSGD() + algorithm.optimizer + .setNumIterations(params.numIterations) + .setStepSize(params.stepSize) + .setUpdater(updater) + .setRegParam(params.regParam) + algorithm.run(training).clearThreshold() + } + + val prediction = model.predict(test.map(_.features)) + val predictionAndLabel = prediction.zip(test.map(_.label)) + + val metrics = new BinaryClassificationMetrics(predictionAndLabel) + + println(s"Test areaUnderPR = ${metrics.areaUnderPR()}.") + println(s"Test areaUnderROC = ${metrics.areaUnderROC()}.") + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala new file mode 100644 index 0000000000000..0bd847d7bab30 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -0,0 +1,161 @@ +/* + * 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.examples.mllib + +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{DecisionTree, impurity} +import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.model.DecisionTreeModel +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD + +/** + * An example runner for decision tree. Run with + * {{{ + * ./bin/spark-example org.apache.spark.examples.mllib.DecisionTreeRunner [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object DecisionTreeRunner { + + object ImpurityType extends Enumeration { + type ImpurityType = Value + val Gini, Entropy, Variance = Value + } + + import ImpurityType._ + + case class Params( + input: String = null, + algo: Algo = Classification, + maxDepth: Int = 5, + impurity: ImpurityType = Gini, + maxBins: Int = 20) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("DecisionTreeRunner") { + head("DecisionTreeRunner: an example decision tree app.") + opt[String]("algo") + .text(s"algorithm (${Algo.values.mkString(",")}), default: ${defaultParams.algo}") + .action((x, c) => c.copy(algo = Algo.withName(x))) + opt[String]("impurity") + .text(s"impurity type (${ImpurityType.values.mkString(",")}), " + + s"default: ${defaultParams.impurity}") + .action((x, c) => c.copy(impurity = ImpurityType.withName(x))) + opt[Int]("maxDepth") + .text(s"max depth of the tree, default: ${defaultParams.maxDepth}") + .action((x, c) => c.copy(maxDepth = x)) + opt[Int]("maxBins") + .text(s"max number of bins, default: ${defaultParams.maxBins}") + .action((x, c) => c.copy(maxBins = x)) + arg[String]("") + .text("input paths to labeled examples in dense format (label,f0 f1 f2 ...)") + .required() + .action((x, c) => c.copy(input = x)) + checkConfig { params => + if (params.algo == Classification && + (params.impurity == Gini || params.impurity == Entropy)) { + success + } else if (params.algo == Regression && params.impurity == Variance) { + success + } else { + failure(s"Algo ${params.algo} is not compatible with impurity ${params.impurity}.") + } + } + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName("DecisionTreeRunner") + val sc = new SparkContext(conf) + + // Load training data and cache it. + val examples = MLUtils.loadLabeledData(sc, params.input).cache() + + val splits = examples.randomSplit(Array(0.8, 0.2)) + val training = splits(0).cache() + val test = splits(1).cache() + + val numTraining = training.count() + val numTest = test.count() + + println(s"numTraining = $numTraining, numTest = $numTest.") + + examples.unpersist(blocking = false) + + val impurityCalculator = params.impurity match { + case Gini => impurity.Gini + case Entropy => impurity.Entropy + case Variance => impurity.Variance + } + + val strategy = new Strategy(params.algo, impurityCalculator, params.maxDepth, params.maxBins) + val model = DecisionTree.train(training, strategy) + + if (params.algo == Classification) { + val accuracy = accuracyScore(model, test) + println(s"Test accuracy = $accuracy.") + } + + if (params.algo == Regression) { + val mse = meanSquaredError(model, test) + println(s"Test mean squared error = $mse.") + } + + sc.stop() + } + + /** + * Calculates the classifier accuracy. + */ + private def accuracyScore( + model: DecisionTreeModel, + data: RDD[LabeledPoint], + threshold: Double = 0.5): Double = { + def predictedValue(features: Vector): Double = { + if (model.predict(features) < threshold) 0.0 else 1.0 + } + val correctCount = data.filter(y => predictedValue(y.features) == y.label).count() + val count = data.count() + correctCount.toDouble / count + } + + /** + * Calculates the mean squared error for regression. + */ + private def meanSquaredError(tree: DecisionTreeModel, data: RDD[LabeledPoint]): Double = { + data.map { y => + val err = tree.predict(y.features) - y.label + err * err + }.mean() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala new file mode 100644 index 0000000000000..f96bc1bf00b92 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala @@ -0,0 +1,109 @@ +/* + * 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.examples.mllib + +import org.apache.log4j.{Level, Logger} +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.clustering.KMeans +import org.apache.spark.mllib.linalg.Vectors + +/** + * An example k-means app. Run with + * {{{ + * ./bin/spark-example org.apache.spark.examples.mllib.DenseKMeans [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object DenseKMeans { + + object InitializationMode extends Enumeration { + type InitializationMode = Value + val Random, Parallel = Value + } + + import InitializationMode._ + + case class Params( + input: String = null, + k: Int = -1, + numIterations: Int = 10, + initializationMode: InitializationMode = Parallel) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("DenseKMeans") { + head("DenseKMeans: an example k-means app for dense data.") + opt[Int]('k', "k") + .required() + .text(s"number of clusters, required") + .action((x, c) => c.copy(k = x)) + opt[Int]("numIterations") + .text(s"number of iterations, default; ${defaultParams.numIterations}") + .action((x, c) => c.copy(numIterations = x)) + opt[String]("initMode") + .text(s"initialization mode (${InitializationMode.values.mkString(",")}), " + + s"default: ${defaultParams.initializationMode}") + .action((x, c) => c.copy(initializationMode = InitializationMode.withName(x))) + arg[String]("") + .text("input paths to examples") + .required() + .action((x, c) => c.copy(input = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"DenseKMeans with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val examples = sc.textFile(params.input).map { line => + Vectors.dense(line.split(' ').map(_.toDouble)) + }.cache() + + val numExamples = examples.count() + + println(s"numExamples = $numExamples.") + + val initMode = params.initializationMode match { + case Random => KMeans.RANDOM + case Parallel => KMeans.K_MEANS_PARALLEL + } + + val model = new KMeans() + .setInitializationMode(initMode) + .setK(params.k) + .setMaxIterations(params.numIterations) + .run(examples) + + val cost = model.computeCost(examples) + + println(s"Total cost = $cost.") + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala new file mode 100644 index 0000000000000..1723ca6931021 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala @@ -0,0 +1,125 @@ +/* + * 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.examples.mllib + +import org.apache.log4j.{Level, Logger} +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.regression.LinearRegressionWithSGD +import org.apache.spark.mllib.util.{MulticlassLabelParser, MLUtils} +import org.apache.spark.mllib.optimization.{SimpleUpdater, SquaredL2Updater, L1Updater} + +/** + * An example app for linear regression. Run with + * {{{ + * ./bin/run-example org.apache.spark.examples.mllib.LinearRegression + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object LinearRegression extends App { + + object RegType extends Enumeration { + type RegType = Value + val NONE, L1, L2 = Value + } + + import RegType._ + + case class Params( + input: String = null, + numIterations: Int = 100, + stepSize: Double = 1.0, + regType: RegType = L2, + regParam: Double = 0.1) + + val defaultParams = Params() + + val parser = new OptionParser[Params]("LinearRegression") { + head("LinearRegression: an example app for linear regression.") + opt[Int]("numIterations") + .text("number of iterations") + .action((x, c) => c.copy(numIterations = x)) + opt[Double]("stepSize") + .text(s"initial step size, default: ${defaultParams.stepSize}") + .action((x, c) => c.copy(stepSize = x)) + opt[String]("regType") + .text(s"regularization type (${RegType.values.mkString(",")}), " + + s"default: ${defaultParams.regType}") + .action((x, c) => c.copy(regType = RegType.withName(x))) + opt[Double]("regParam") + .text(s"regularization parameter, default: ${defaultParams.regParam}") + arg[String]("") + .required() + .text("input paths to labeled examples in LIBSVM format") + .action((x, c) => c.copy(input = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + sys.exit(1) + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"LinearRegression with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val examples = MLUtils.loadLibSVMData(sc, params.input, MulticlassLabelParser).cache() + + val splits = examples.randomSplit(Array(0.8, 0.2)) + val training = splits(0).cache() + val test = splits(1).cache() + + val numTraining = training.count() + val numTest = test.count() + println(s"Training: $numTraining, test: $numTest.") + + examples.unpersist(blocking = false) + + val updater = params.regType match { + case NONE => new SimpleUpdater() + case L1 => new L1Updater() + case L2 => new SquaredL2Updater() + } + + val algorithm = new LinearRegressionWithSGD() + algorithm.optimizer + .setNumIterations(params.numIterations) + .setStepSize(params.stepSize) + .setUpdater(updater) + .setRegParam(params.regParam) + + val model = algorithm.run(training) + + val prediction = model.predict(test.map(_.features)) + val predictionAndLabel = prediction.zip(test.map(_.label)) + + val loss = predictionAndLabel.map { case (p, l) => + val err = p - l + err * err + }.reduce(_ + _) + val rmse = math.sqrt(loss / numTest) + + println(s"Test RMSE = $rmse.") + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala new file mode 100644 index 0000000000000..703f02255b94b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala @@ -0,0 +1,131 @@ +/* + * 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.examples.mllib + +import com.esotericsoftware.kryo.Kryo +import org.apache.log4j.{Level, Logger} +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.recommendation.{ALS, MatrixFactorizationModel, Rating} +import org.apache.spark.rdd.RDD +import org.apache.spark.serializer.{KryoSerializer, KryoRegistrator} + +/** + * An example app for ALS on MovieLens data (http://grouplens.org/datasets/movielens/). + */ +object MovieLensALS { + + class ALSRegistrator extends KryoRegistrator { + override def registerClasses(kryo: Kryo) { + kryo.register(classOf[Rating]) + } + } + + case class Params( + input: String = null, + kryo: Boolean = false, + numIterations: Int = 20, + lambda: Double = 1.0, + rank: Int = 10) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("MovieLensALS") { + head("MovieLensALS: an example app for ALS on MovieLens data.") + opt[Int]("rank") + .text(s"rank, default: ${defaultParams.rank}}") + .action((x, c) => c.copy(rank = x)) + opt[Int]("numIterations") + .text(s"number of iterations, default: ${defaultParams.numIterations}") + .action((x, c) => c.copy(numIterations = x)) + opt[Double]("lambda") + .text(s"lambda (smoothing constant), default: ${defaultParams.lambda}") + .action((x, c) => c.copy(lambda = x)) + opt[Unit]("kryo") + .text(s"use Kryo serialization") + .action((_, c) => c.copy(kryo = true)) + arg[String]("") + .required() + .text("input paths to a MovieLens dataset of ratings") + .action((x, c) => c.copy(input = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + System.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"MovieLensALS with $params") + if (params.kryo) { + conf.set("spark.serializer", classOf[KryoSerializer].getName) + .set("spark.kryo.registrator", classOf[ALSRegistrator].getName) + .set("spark.kryoserializer.buffer.mb", "8") + } + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val ratings = sc.textFile(params.input).map { line => + val fields = line.split("::") + Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble) + }.cache() + + val numRatings = ratings.count() + val numUsers = ratings.map(_.user).distinct().count() + val numMovies = ratings.map(_.product).distinct().count() + + println(s"Got $numRatings ratings from $numUsers users on $numMovies movies.") + + val splits = ratings.randomSplit(Array(0.8, 0.2)) + val training = splits(0).cache() + val test = splits(1).cache() + + val numTraining = training.count() + val numTest = test.count() + println(s"Training: $numTraining, test: $numTest.") + + ratings.unpersist(blocking = false) + + val model = new ALS() + .setRank(params.rank) + .setIterations(params.numIterations) + .setLambda(params.lambda) + .run(training) + + val rmse = computeRmse(model, test, numTest) + + println(s"Test RMSE = $rmse.") + + sc.stop() + } + + /** Compute RMSE (Root Mean Squared Error). */ + def computeRmse(model: MatrixFactorizationModel, data: RDD[Rating], n: Long) = { + val predictions: RDD[Rating] = model.predict(data.map(x => (x.user, x.product))) + val predictionsAndRatings = predictions.map(x => ((x.user, x.product), x.rating)) + .join(data.map(x => ((x.user, x.product), x.rating))) + .values + math.sqrt(predictionsAndRatings.map(x => (x._1 - x._2) * (x._1 - x._2)).reduce(_ + _) / n) + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala new file mode 100644 index 0000000000000..25b6768b8d72b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala @@ -0,0 +1,102 @@ +/* + * 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.examples.mllib + +import org.apache.log4j.{Level, Logger} +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.classification.NaiveBayes +import org.apache.spark.mllib.util.{MLUtils, MulticlassLabelParser} + +/** + * An example naive Bayes app. Run with + * {{{ + * ./bin/spark-example org.apache.spark.examples.mllib.SparseNaiveBayes [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object SparseNaiveBayes { + + case class Params( + input: String = null, + minPartitions: Int = 0, + numFeatures: Int = -1, + lambda: Double = 1.0) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("SparseNaiveBayes") { + head("SparseNaiveBayes: an example naive Bayes app for LIBSVM data.") + opt[Int]("numPartitions") + .text("min number of partitions") + .action((x, c) => c.copy(minPartitions = x)) + opt[Int]("numFeatures") + .text("number of features") + .action((x, c) => c.copy(numFeatures = x)) + opt[Double]("lambda") + .text(s"lambda (smoothing constant), default: ${defaultParams.lambda}") + .action((x, c) => c.copy(lambda = x)) + arg[String]("") + .text("input paths to labeled examples in LIBSVM format") + .required() + .action((x, c) => c.copy(input = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"SparseNaiveBayes with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val minPartitions = + if (params.minPartitions > 0) params.minPartitions else sc.defaultMinPartitions + + val examples = MLUtils.loadLibSVMData(sc, params.input, MulticlassLabelParser, + params.numFeatures, minPartitions) + // Cache examples because it will be used in both training and evaluation. + examples.cache() + + val splits = examples.randomSplit(Array(0.8, 0.2)) + val training = splits(0) + val test = splits(1) + + val numTraining = training.count() + val numTest = test.count() + + println(s"numTraining = $numTraining, numTest = $numTest.") + + val model = new NaiveBayes().setLambda(params.lambda).run(training) + + val prediction = model.predict(test.map(_.features)) + val predictionAndLabel = prediction.zip(test.map(_.label)) + val accuracy = predictionAndLabel.filter(x => x._1 == x._2).count().toDouble / numTest + + println(s"Test accuracy = $accuracy.") + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala index 39e71cdab4328..3cd9cb743e309 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -35,20 +35,16 @@ import org.apache.spark.mllib.linalg.Vectors */ object TallSkinnyPCA { def main(args: Array[String]) { - if (args.length != 2) { - System.err.println("Usage: TallSkinnyPCA ") + if (args.length != 1) { + System.err.println("Usage: TallSkinnyPCA ") System.exit(1) } - val conf = new SparkConf() - .setMaster(args(0)) - .setAppName("TallSkinnyPCA") - .setSparkHome(System.getenv("SPARK_HOME")) - .setJars(SparkContext.jarOfClass(this.getClass).toSeq) + val conf = new SparkConf().setAppName("TallSkinnyPCA") val sc = new SparkContext(conf) // Load and parse the data file. - val rows = sc.textFile(args(1)).map { line => + val rows = sc.textFile(args(0)).map { line => val values = line.split(' ').map(_.toDouble) Vectors.dense(values) } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala index 2b7de2acc65d6..4d6690318615a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -35,20 +35,16 @@ import org.apache.spark.mllib.linalg.Vectors */ object TallSkinnySVD { def main(args: Array[String]) { - if (args.length != 2) { - System.err.println("Usage: TallSkinnySVD ") + if (args.length != 1) { + System.err.println("Usage: TallSkinnySVD ") System.exit(1) } - val conf = new SparkConf() - .setMaster(args(0)) - .setAppName("TallSkinnySVD") - .setSparkHome(System.getenv("SPARK_HOME")) - .setJars(SparkContext.jarOfClass(this.getClass).toSeq) + val conf = new SparkConf().setAppName("TallSkinnySVD") val sc = new SparkContext(conf) // Load and parse the data file. - val rows = sc.textFile(args(1)).map { line => + val rows = sc.textFile(args(0)).map { line => val values = line.split(' ').map(_.toDouble) Vectors.dense(values) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 4f9eaacf67fe4..780e8bae42b84 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -17,11 +17,10 @@ package org.apache.spark.mllib.classification -import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.{DataValidators, MLUtils} +import org.apache.spark.mllib.util.DataValidators import org.apache.spark.rdd.RDD /** @@ -183,19 +182,4 @@ object LogisticRegressionWithSGD { numIterations: Int): LogisticRegressionModel = { train(input, numIterations, 1.0, 1.0) } - - def main(args: Array[String]) { - if (args.length != 4) { - println("Usage: LogisticRegression " + - "") - System.exit(1) - } - val sc = new SparkContext(args(0), "LogisticRegression") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LogisticRegressionWithSGD.train(data, args(3).toInt, args(2).toDouble) - println("Weights: " + model.weights) - println("Intercept: " + model.intercept) - - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index 18658850a2f64..f6f62ce2de04e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -20,11 +20,10 @@ package org.apache.spark.mllib.classification import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} import org.apache.spark.annotation.Experimental -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.Logging import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD /** @@ -158,23 +157,4 @@ object NaiveBayes { def train(input: RDD[LabeledPoint], lambda: Double): NaiveBayesModel = { new NaiveBayes(lambda).run(input) } - - def main(args: Array[String]) { - if (args.length != 2 && args.length != 3) { - println("Usage: NaiveBayes []") - System.exit(1) - } - val sc = new SparkContext(args(0), "NaiveBayes") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = if (args.length == 2) { - NaiveBayes.train(data) - } else { - NaiveBayes.train(data, args(2).toDouble) - } - - println("Pi\n: " + model.pi) - println("Theta:\n" + model.theta) - - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 956654b1fe90a..81b126717e9a4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -17,11 +17,10 @@ package org.apache.spark.mllib.classification -import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.{DataValidators, MLUtils} +import org.apache.spark.mllib.util.DataValidators import org.apache.spark.rdd.RDD /** @@ -183,19 +182,4 @@ object SVMWithSGD { def train(input: RDD[LabeledPoint], numIterations: Int): SVMModel = { train(input, numIterations, 1.0, 1.0, 1.0) } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: SVM ") - System.exit(1) - } - val sc = new SparkContext(args(0), "SVM") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = SVMWithSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) - - println("Weights: " + model.weights) - println("Intercept: " + model.intercept) - - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index dee9ef07e41ed..a64c5d44be406 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -21,8 +21,7 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{DenseVector => BDV, Vector => BV, norm => breezeNorm} -import org.apache.spark.annotation.Experimental -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.Logging import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLUtils @@ -396,28 +395,6 @@ object KMeans { v2: BreezeVectorWithNorm): Double = { MLUtils.fastSquaredDistance(v1.vector, v1.norm, v2.vector, v2.norm) } - - @Experimental - def main(args: Array[String]) { - if (args.length < 4) { - println("Usage: KMeans []") - System.exit(1) - } - val (master, inputFile, k, iters) = (args(0), args(1), args(2).toInt, args(3).toInt) - val runs = if (args.length >= 5) args(4).toInt else 1 - val sc = new SparkContext(master, "KMeans") - val data = sc.textFile(inputFile) - .map(line => Vectors.dense(line.split(' ').map(_.toDouble))) - .cache() - val model = KMeans.train(data, k, iters, runs) - val cost = model.computeCost(data) - println("Cluster centers:") - for (c <- model.clusterCenters) { - println(" " + c) - } - println("Cost: " + cost) - System.exit(0) - } } /** 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 60fb73f2b5be5..2a77e1a9efb2e 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 @@ -23,15 +23,13 @@ import scala.util.Random import scala.util.Sorting import scala.util.hashing.byteswap32 -import com.esotericsoftware.kryo.Kryo import org.jblas.{DoubleMatrix, SimpleBlas, Solve} import org.apache.spark.annotation.Experimental import org.apache.spark.broadcast.Broadcast -import org.apache.spark.{Logging, HashPartitioner, Partitioner, SparkContext, SparkConf} +import org.apache.spark.{Logging, HashPartitioner, Partitioner} import org.apache.spark.storage.StorageLevel import org.apache.spark.rdd.RDD -import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.SparkContext._ import org.apache.spark.util.Utils @@ -707,45 +705,4 @@ object ALS { : MatrixFactorizationModel = { trainImplicit(ratings, rank, iterations, 0.01, -1, 1.0) } - - private class ALSRegistrator extends KryoRegistrator { - override def registerClasses(kryo: Kryo) { - kryo.register(classOf[Rating]) - } - } - - def main(args: Array[String]) { - if (args.length < 5 || args.length > 9) { - println("Usage: ALS " + - "[] [] [] []") - System.exit(1) - } - val (master, ratingsFile, rank, iters, outputDir) = - (args(0), args(1), args(2).toInt, args(3).toInt, args(4)) - val lambda = if (args.length >= 6) args(5).toDouble else 0.01 - val implicitPrefs = if (args.length >= 7) args(6).toBoolean else false - val alpha = if (args.length >= 8) args(7).toDouble else 1 - val blocks = if (args.length == 9) args(8).toInt else -1 - val conf = new SparkConf() - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", classOf[ALSRegistrator].getName) - .set("spark.kryo.referenceTracking", "false") - .set("spark.kryoserializer.buffer.mb", "8") - .set("spark.locality.wait", "10000") - val sc = new SparkContext(master, "ALS", conf) - - val ratings = sc.textFile(ratingsFile).map { line => - val fields = line.split(',') - Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble) - } - val model = new ALS(rank = rank, iterations = iters, lambda = lambda, - numBlocks = blocks, implicitPrefs = implicitPrefs, alpha = alpha).run(ratings) - - model.userFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") } - .saveAsTextFile(outputDir + "/userFeatures") - model.productFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") } - .saveAsTextFile(outputDir + "/productFeatures") - println("Final user/product features written to " + outputDir) - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index 5f0812fd2e0eb..0e6fb1b1caa41 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -17,10 +17,8 @@ package org.apache.spark.mllib.regression -import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ -import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD /** @@ -173,19 +171,4 @@ object LassoWithSGD { numIterations: Int): LassoModel = { train(input, numIterations, 1.0, 1.0, 1.0) } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: Lasso ") - System.exit(1) - } - val sc = new SparkContext(args(0), "Lasso") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LassoWithSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) - - println("Weights: " + model.weights) - println("Intercept: " + model.intercept) - - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 228fa8db3e721..1532ff90d846d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -17,11 +17,9 @@ package org.apache.spark.mllib.regression -import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ -import org.apache.spark.mllib.util.MLUtils /** * Regression model trained using LinearRegression. @@ -156,18 +154,4 @@ object LinearRegressionWithSGD { numIterations: Int): LinearRegressionModel = { train(input, numIterations, 1.0, 1.0) } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: LinearRegression ") - System.exit(1) - } - val sc = new SparkContext(args(0), "LinearRegression") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LinearRegressionWithSGD.train(data, args(3).toInt, args(2).toDouble) - println("Weights: " + model.weights) - println("Intercept: " + model.intercept) - - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index e702027c7c170..5f7e25a9b8be1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -17,10 +17,8 @@ package org.apache.spark.mllib.regression -import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ -import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.linalg.Vector /** @@ -170,21 +168,4 @@ object RidgeRegressionWithSGD { numIterations: Int): RidgeRegressionModel = { train(input, numIterations, 1.0, 1.0, 1.0) } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: RidgeRegression " + - " ") - System.exit(1) - } - val sc = new SparkContext(args(0), "RidgeRegression") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = RidgeRegressionWithSGD.train(data, args(4).toInt, args(2).toDouble, - args(3).toDouble) - - println("Weights: " + model.weights) - println("Intercept: " + model.intercept) - - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index f68076f426259..59ed01debf150 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -18,18 +18,16 @@ package org.apache.spark.mllib.tree import org.apache.spark.annotation.Experimental -import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.SparkContext._ +import org.apache.spark.Logging import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ -import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Impurity, Variance} +import org.apache.spark.mllib.tree.impurity.Impurity import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom -import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * :: Experimental :: @@ -1028,129 +1026,4 @@ object DecisionTree extends Serializable with Logging { throw new UnsupportedOperationException("approximate histogram not supported yet.") } } - - private val usage = """ - Usage: DecisionTreeRunner [slices] --algo --trainDataDir path --testDataDir path --maxDepth num [--impurity ] [--maxBins num] - """ - - def main(args: Array[String]) { - - if (args.length < 2) { - System.err.println(usage) - System.exit(1) - } - - val sc = new SparkContext(args(0), "DecisionTree") - - val argList = args.toList.drop(1) - type OptionMap = Map[Symbol, Any] - - def nextOption(map : OptionMap, list: List[String]): OptionMap = { - list match { - case Nil => map - case "--algo" :: string :: tail => nextOption(map ++ Map('algo -> string), tail) - case "--impurity" :: string :: tail => nextOption(map ++ Map('impurity -> string), tail) - case "--maxDepth" :: string :: tail => nextOption(map ++ Map('maxDepth -> string), tail) - case "--maxBins" :: string :: tail => nextOption(map ++ Map('maxBins -> string), tail) - case "--trainDataDir" :: string :: tail => nextOption(map ++ Map('trainDataDir -> string) - , tail) - case "--testDataDir" :: string :: tail => nextOption(map ++ Map('testDataDir -> string), - tail) - case string :: Nil => nextOption(map ++ Map('infile -> string), list.tail) - case option :: tail => logError("Unknown option " + option) - sys.exit(1) - } - } - val options = nextOption(Map(), argList) - logDebug(options.toString()) - - // Load training data. - val trainData = loadLabeledData(sc, options.get('trainDataDir).get.toString) - - // Identify the type of algorithm. - val algoStr = options.get('algo).get.toString - val algo = algoStr match { - case "Classification" => Classification - case "Regression" => Regression - } - - // Identify the type of impurity. - val impurityStr = options.getOrElse('impurity, - if (algo == Classification) "Gini" else "Variance").toString - val impurity = impurityStr match { - case "Gini" => Gini - case "Entropy" => Entropy - case "Variance" => Variance - } - - val maxDepth = options.getOrElse('maxDepth, "1").toString.toInt - val maxBins = options.getOrElse('maxBins, "100").toString.toInt - - val strategy = new Strategy(algo, impurity, maxDepth, maxBins) - val model = DecisionTree.train(trainData, strategy) - - // Load test data. - val testData = loadLabeledData(sc, options.get('testDataDir).get.toString) - - // Measure algorithm accuracy - if (algo == Classification) { - val accuracy = accuracyScore(model, testData) - logDebug("accuracy = " + accuracy) - } - - if (algo == Regression) { - val mse = meanSquaredError(model, testData) - logDebug("mean square error = " + mse) - } - - sc.stop() - } - - /** - * Load labeled data from a file. The data format used here is - * , ..., - * where , are feature values in Double and is the corresponding label as Double. - * - * @param sc SparkContext - * @param dir Directory to the input data files. - * @return An RDD of LabeledPoint. Each labeled point has two elements: the first element is - * the label, and the second element represents the feature values (an array of Double). - */ - private def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { - sc.textFile(dir).map { line => - val parts = line.trim().split(",") - val label = parts(0).toDouble - val features = Vectors.dense(parts.slice(1,parts.length).map(_.toDouble)) - LabeledPoint(label, features) - } - } - - // TODO: Port this method to a generic metrics package. - /** - * Calculates the classifier accuracy. - */ - private def accuracyScore(model: DecisionTreeModel, data: RDD[LabeledPoint], - threshold: Double = 0.5): Double = { - def predictedValue(features: Vector) = { - if (model.predict(features) < threshold) 0.0 else 1.0 - } - val correctCount = data.filter(y => predictedValue(y.features) == y.label).count() - val count = data.count() - logDebug("correct prediction count = " + correctCount) - logDebug("data count = " + count) - correctCount.toDouble / count - } - - // TODO: Port this method to a generic metrics package - /** - * Calculates the mean squared error for regression. - */ - private def meanSquaredError(tree: DecisionTreeModel, data: RDD[LabeledPoint]): Double = { - data.map { y => - val err = tree.predict(y.features) - y.label - err * err - }.mean() - } } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a9504f3926082..51f7335111166 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -426,7 +426,8 @@ object SparkBuild extends Build { exclude("io.netty", "netty") exclude("jline","jline") exclude("org.apache.cassandra.deps", "avro") - excludeAll(excludeSLF4J) + excludeAll(excludeSLF4J), + "com.github.scopt" %% "scopt" % "3.2.0" ) ) ++ assemblySettings ++ extraAssemblySettings From f04bcaf6fc3379f54f722caf70b221a82b4fcc12 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 29 Apr 2014 00:53:32 -0700 Subject: [PATCH 207/641] HOTFIX: minor change to release script --- dev/create-release/create-release.sh | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 44da8f7cbf53f..01064cdc4de2a 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -115,9 +115,9 @@ make_binary_release "hadoop2" "--with-yarn --hadoop 2.2.0" # Copy data echo "Copying release tarballs" -ssh $USER_NAME@people.apache.org \ - mkdir /home/$USER_NAME/public_html/spark-$RELEASE_VERSION-$RC_NAME rc_folder=spark-$RELEASE_VERSION-$RC_NAME +ssh $USER_NAME@people.apache.org \ + mkdir /home/$USER_NAME/public_html/$rc_folder scp spark-* \ $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_folder/ @@ -127,6 +127,8 @@ cd docs PRODUCTION=1 jekyll build echo "Copying release documentation" rc_docs_folder=${rc_folder}-docs +ssh $USER_NAME@people.apache.org \ + mkdir /home/$USER_NAME/public_html/$rc_docs_folder rsync -r _site/* $USER_NAME@people.apache.org /home/$USER_NAME/public_html/$rc_docs_folder echo "Release $RELEASE_VERSION completed:" From 9a1184a8a9fa679b5ba8cf376b2c71ed1fb6e961 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 29 Apr 2014 00:59:38 -0700 Subject: [PATCH 208/641] HOTFIX: minor change to release script --- dev/create-release/create-release.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 01064cdc4de2a..33552a74920a7 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -129,7 +129,7 @@ echo "Copying release documentation" rc_docs_folder=${rc_folder}-docs ssh $USER_NAME@people.apache.org \ mkdir /home/$USER_NAME/public_html/$rc_docs_folder -rsync -r _site/* $USER_NAME@people.apache.org /home/$USER_NAME/public_html/$rc_docs_folder +rsync -r _site/* $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_docs_folder echo "Release $RELEASE_VERSION completed:" echo "Git tag:\t $GIT_TAG" From 8db0f7e28f5f0330a3344705ff48d8e7b97c383f Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 29 Apr 2014 09:19:48 -0500 Subject: [PATCH 209/641] SPARK-1557 Set permissions on event log files/directories This adds minimal setting of event log directory/files permissions. To have a secure environment the user must manually create the top level event log directory and set permissions up. We can add logic to do that automatically later if we want. Author: Thomas Graves Closes #538 from tgravescs/SPARK-1557 and squashes the following commits: e471d8e [Thomas Graves] rework d8b6620 [Thomas Graves] update use of octal 3ca9b79 [Thomas Graves] Updated based on comments 5a09709 [Thomas Graves] add in missing import 3150ed6 [Thomas Graves] SPARK-1557 Set permissions on event log files/directories --- .../scheduler/EventLoggingListener.scala | 6 ++++- .../org/apache/spark/util/FileLogger.scala | 22 ++++++++++++++----- docs/security.md | 2 ++ 3 files changed, 24 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 2fe65cd944b67..d822a8e55111a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -21,6 +21,7 @@ import scala.collection.mutable import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.permission.FsPermission import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkConf, SparkContext} @@ -54,7 +55,7 @@ private[spark] class EventLoggingListener( private val logger = new FileLogger(logDir, conf, hadoopConfiguration, outputBufferSize, shouldCompress, - shouldOverwrite) + shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) /** * Begin logging events. @@ -124,6 +125,9 @@ private[spark] object EventLoggingListener extends Logging { val SPARK_VERSION_PREFIX = "SPARK_VERSION_" val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" + val LOG_FILE_PERMISSIONS: FsPermission = + FsPermission.createImmutable(Integer.parseInt("770", 8).toShort) + // A cache for compression codecs to avoid creating the same codec many times private val codecMap = new mutable.HashMap[String, CompressionCodec] diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 1ed3b70bb24fd..0965e0f0f7828 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -24,6 +24,7 @@ import java.util.Date import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} +import org.apache.hadoop.fs.permission.FsPermission import org.apache.spark.{Logging, SparkConf} import org.apache.spark.io.CompressionCodec @@ -42,7 +43,8 @@ private[spark] class FileLogger( hadoopConfiguration: Configuration, outputBufferSize: Int = 8 * 1024, // 8 KB compress: Boolean = false, - overwrite: Boolean = true) + overwrite: Boolean = true, + dirPermissions: Option[FsPermission] = None) extends Logging { private val dateFormat = new ThreadLocal[SimpleDateFormat]() { @@ -79,16 +81,25 @@ private[spark] class FileLogger( if (!fileSystem.mkdirs(path)) { throw new IOException("Error in creating log directory: %s".format(logDir)) } + if (dirPermissions.isDefined) { + val fsStatus = fileSystem.getFileStatus(path) + if (fsStatus.getPermission().toShort() != dirPermissions.get.toShort) { + fileSystem.setPermission(path, dirPermissions.get) + } + } } /** * Create a new writer for the file identified by the given path. + * If the permissions are not passed in, it will default to use the permissions + * (dirpermissions) used when class was instantiated. */ - private def createWriter(fileName: String): PrintWriter = { + private def createWriter(fileName: String, perms: Option[FsPermission] = None): PrintWriter = { val logPath = logDir + "/" + fileName val uri = new URI(logPath) val defaultFs = FileSystem.getDefaultUri(hadoopConfiguration).getScheme val isDefaultLocal = (defaultFs == null || defaultFs == "file") + val path = new Path(logPath) /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). * Therefore, for local files, use FileOutputStream instead. */ @@ -97,11 +108,11 @@ private[spark] class FileLogger( // Second parameter is whether to append new FileOutputStream(uri.getPath, !overwrite) } else { - val path = new Path(logPath) hadoopDataStream = Some(fileSystem.create(path, overwrite)) hadoopDataStream.get } + perms.orElse(dirPermissions).foreach {p => fileSystem.setPermission(path, p)} val bstream = new BufferedOutputStream(dstream, outputBufferSize) val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream new PrintWriter(cstream) @@ -152,15 +163,16 @@ private[spark] class FileLogger( /** * Start a writer for a new file, closing the existing one if it exists. * @param fileName Name of the new file, defaulting to the file index if not provided. + * @param perms Permissions to put on the new file. */ - def newFile(fileName: String = "") { + def newFile(fileName: String = "", perms: Option[FsPermission] = None) { fileIndex += 1 writer.foreach(_.close()) val name = fileName match { case "" => fileIndex.toString case _ => fileName } - writer = Some(createWriter(name)) + writer = Some(createWriter(name, perms)) } /** diff --git a/docs/security.md b/docs/security.md index 9e4218fbcfe7d..90c69915f517f 100644 --- a/docs/security.md +++ b/docs/security.md @@ -7,6 +7,8 @@ Spark currently supports authentication via a shared secret. Authentication can The Spark UI can also be secured by using javax servlet filters. A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view acls to make sure they are authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls' control the behavior of the acls. Note that the person who started the application always has view access to the UI. +If your applications are using event logging, the directory where the event logs go (`spark.eventLog.dir`) should be manually created and have the proper permissions set on it. If you want those log files secure, the permissions should be set to drwxrwxrwxt for that directory. The owner of the directory should be the super user who is running the history server and the group permissions should be restricted to super user group. This will allow all users to write to the directory but will prevent unprivileged users from removing or renaming a file unless they own the file or directory. The event log files will be created by Spark with permissions such that only the user and group have read and write access. + For Spark on Yarn deployments, configuring `spark.authenticate` to true will automatically handle generating and distributing the shared secret. Each application will use a unique shared secret. The Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. If an authentication filter is enabled, the acls controls can be used by control which users can via the Spark UI. For other types of Spark deployments, the spark config `spark.authenticate.secret` should be configured on each of the nodes. This secret will be used by all the Master/Workers and applications. The UI can be secured using a javax servlet filter installed via `spark.ui.filters`. If an authentication filter is enabled, the acls controls can be used by control which users can via the Spark UI. From 7d1505841069c6ecc3fa7e4896db535f18e4ce84 Mon Sep 17 00:00:00 2001 From: witgo Date: Tue, 29 Apr 2014 11:30:47 -0700 Subject: [PATCH 210/641] SPARK-1509: add zipWithIndex zipWithUniqueId methods to java api Author: witgo Closes #423 from witgo/zipWithIndex and squashes the following commits: 039ec04 [witgo] Merge branch 'master' of https://github.com/apache/spark into zipWithIndex 24d74c9 [witgo] review commit 763a5e4 [witgo] Merge branch 'master' of https://github.com/apache/spark into zipWithIndex 59747d1 [witgo] review commit 7bf4d06 [witgo] Merge branch 'master' of https://github.com/apache/spark into zipWithIndex daa8f84 [witgo] review commit 4070613 [witgo] Merge branch 'master' of https://github.com/apache/spark into zipWithIndex 18e6c97 [witgo] java api zipWithIndex test 11e2e7f [witgo] add zipWithIndex zipWithUniqueId methods to java api --- .../apache/spark/api/java/JavaRDDLike.scala | 22 ++++++++++++- .../java/org/apache/spark/JavaAPISuite.java | 31 ++++++++++++++----- 2 files changed, 45 insertions(+), 8 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 574a98636a619..af06d1dca94f4 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 @@ -18,7 +18,7 @@ package org.apache.spark.api.java import java.util.{Comparator, List => JList, Iterator => JIterator} -import java.lang.{Iterable => JIterable} +import java.lang.{Iterable => JIterable, Long => JLong} import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -264,6 +264,26 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { rdd.zipPartitions(other.rdd)(fn)(other.classTag, fakeClassTag[V]))(fakeClassTag[V]) } + /** + * Zips this RDD with generated unique Long ids. Items in the kth partition will get ids k, n+k, + * 2*n+k, ..., where n is the number of partitions. So there may exist gaps, but this method + * won't trigger a spark job, which is different from [[org.apache.spark.rdd.RDD#zipWithIndex]]. + */ + def zipWithUniqueId(): JavaPairRDD[T, JLong] = { + JavaPairRDD.fromRDD(rdd.zipWithUniqueId()).asInstanceOf[JavaPairRDD[T, JLong]] + } + + /** + * Zips this RDD with its element indices. The ordering is first based on the partition index + * and then the ordering of items within each partition. So the first item in the first + * partition gets index 0, and the last item in the last partition receives the largest index. + * This is similar to Scala's zipWithIndex but it uses Long instead of Int as the index type. + * This method needs to trigger a spark job when this RDD contains more than one partitions. + */ + def zipWithIndex(): JavaPairRDD[T, JLong] = { + JavaPairRDD.fromRDD(rdd.zipWithIndex()).asInstanceOf[JavaPairRDD[T, JLong]] + } + // Actions (launch a job to return a value to the user program) /** diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 76c6f5af82a5e..c3e03cea917b3 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -182,13 +182,30 @@ public void call(String s) { Assert.assertEquals(2, foreachCalls); } - @Test - public void toLocalIterator() { - List correct = Arrays.asList(1, 2, 3, 4); - JavaRDD rdd = sc.parallelize(correct); - List result = Lists.newArrayList(rdd.toLocalIterator()); - Assert.assertTrue(correct.equals(result)); - } + @Test + public void toLocalIterator() { + List correct = Arrays.asList(1, 2, 3, 4); + JavaRDD rdd = sc.parallelize(correct); + List result = Lists.newArrayList(rdd.toLocalIterator()); + Assert.assertTrue(correct.equals(result)); + } + + @Test + public void zipWithUniqueId() { + List dataArray = Arrays.asList(1, 2, 3, 4); + JavaPairRDD zip = sc.parallelize(dataArray).zipWithUniqueId(); + JavaRDD indexes = zip.values(); + Assert.assertTrue(new HashSet(indexes.collect()).size() == 4); + } + + @Test + public void zipWithIndex() { + List dataArray = Arrays.asList(1, 2, 3, 4); + JavaPairRDD zip = sc.parallelize(dataArray).zipWithIndex(); + JavaRDD indexes = zip.values(); + List correctIndexes = Arrays.asList(0L, 1L, 2L, 3L); + Assert.assertTrue(indexes.collect().equals(correctIndexes)); + } @SuppressWarnings("unchecked") @Test From bf8d0aa27844b1e58f131d49a3f668d1614ca6e7 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 29 Apr 2014 12:54:02 -0700 Subject: [PATCH 211/641] SPARK-1588. Restore SPARK_YARN_USER_ENV and SPARK_JAVA_OPTS for YARN. Author: Sandy Ryza Closes #586 from sryza/sandy-spark-1588 and squashes the following commits: 35eb38e [Sandy Ryza] Scalify b361684 [Sandy Ryza] SPARK-1588. Restore SPARK_YARN_USER_ENV and SPARK_JAVA_OPTS for YARN. --- .../apache/spark/deploy/yarn/ClientBase.scala | 16 +++++++++++++--- .../spark/deploy/yarn/ExecutorRunnableUtil.scala | 4 ++-- 2 files changed, 15 insertions(+), 5 deletions(-) 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 f2be8217a2f8a..27a518ccda459 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 @@ -263,9 +263,13 @@ trait ClientBase extends Logging { distCacheMgr.setDistFilesEnv(env) distCacheMgr.setDistArchivesEnv(env) - // Allow users to specify some environment variables. - YarnSparkHadoopUtil.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"), - File.pathSeparator) + sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => + // Allow users to specify some environment variables. + YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs, File.pathSeparator) + + // Pass SPARK_YARN_USER_ENV itself to the AM so it can use it to set up executor environments. + env("SPARK_YARN_USER_ENV") = userEnvs + } env } @@ -322,6 +326,12 @@ trait ClientBase extends Logging { JAVA_OPTS += "-XX:CMSIncrementalDutyCycle=10" } + // SPARK_JAVA_OPTS is deprecated, but for backwards compatibility: + sys.env.get("SPARK_JAVA_OPTS").foreach { opts => + sparkConf.set("spark.executor.extraJavaOptions", opts) + sparkConf.set("spark.driver.extraJavaOptions", opts) + } + // TODO: it might be nicer to pass these as an internal environment variable rather than // as Java options, due to complications with string parsing of nested quotes. if (args.amClass == classOf[ExecutorLauncher].getName) { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 7d07f6f68046a..96f8aa93394f5 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -71,8 +71,8 @@ trait ExecutorRunnableUtil extends Logging { /* else { // If no java_opts specified, default to using -XX:+CMSIncrementalMode - // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont - // want to mess with it. + // It might be possible that other modes/config is being done in spark.executor.extraJavaOptions, + // so we dont want to mess with it. // In our expts, using (default) throughput collector has severe perf ramnifications in // multi-tennent machines // The options are based on From d33df1c151f8e982edd7324edc06d8cd3024dd34 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 29 Apr 2014 18:06:45 -0700 Subject: [PATCH 212/641] [SPARK-1674] fix interrupted system call error in pyspark's RDD.pipe `RDD.pipe`'s doctest throws interrupted system call exception on Mac. It can be fixed by wrapping `pipe.stdout.readline` in an iterator. Author: Xiangrui Meng Closes #594 from mengxr/pyspark-pipe and squashes the following commits: cc32ac9 [Xiangrui Meng] fix interrupted system call error in pyspark's RDD.pipe --- python/pyspark/rdd.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index a59778c72130e..3a1c56af5b221 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -537,8 +537,8 @@ def pipe(self, command, env={}): """ Return an RDD created by piping elements to a forked external process. - >>> sc.parallelize([1, 2, 3]).pipe('cat').collect() - ['1', '2', '3'] + >>> sc.parallelize(['1', '2', '', '3']).pipe('cat').collect() + ['1', '2', '', '3'] """ def func(iterator): pipe = Popen(shlex.split(command), env=env, stdin=PIPE, stdout=PIPE) @@ -547,7 +547,7 @@ def pipe_objs(out): out.write(str(obj).rstrip('\n') + '\n') out.close() Thread(target=pipe_objs, args=[pipe.stdin]).start() - return (x.rstrip('\n') for x in pipe.stdout) + return (x.rstrip('\n') for x in iter(pipe.stdout.readline, '')) return self.mapPartitions(func) def foreach(self, f): From 5c0cd5c1a594c181a3f7536639122ab7d97b271b Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Tue, 29 Apr 2014 22:04:34 -0700 Subject: [PATCH 213/641] [SPARK-1646] Micro-optimisation of ALS This change replaces some Scala `for` and `foreach` constructs with `while` constructs. There may be a slight performance gain on the order of 1-2% when training an ALS model. I trained an ALS model on the Movielens 10M-rating dataset repeatedly both with and without these changes. All 7 runs in both columns were done in a Scala `for` loop like this: for (iter <- 0 to 10) { val before = System.currentTimeMillis() val model = ALS.train(rats, 20, 10) val after = System.currentTimeMillis() println("%d ms".format(after-before)) println("rmse %g".format(computeRmse(model, rats, numRatings))) } The timings were done on a multiuser machine, and I stopped one set of timings after 7 had been completed. It would be nice if somebody with dedicated hardware could confirm my timings. After Before 121980 ms 122041 ms 117069 ms 117127 ms 115332 ms 117523 ms 115381 ms 117402 ms 114635 ms 116550 ms 114140 ms 114076 ms 112993 ms 117200 ms Ratios are about 1.0005, 1.0005, 1.019, 1.0175, 1.01671, 0.99944, and 1.03723. I therefore suspect these changes make for a slight performance gain on the order of 1-2%. Author: Tor Myklebust Closes #568 from tmyklebu/alsopt and squashes the following commits: 5ded80f [Tor Myklebust] Fix style. 79595ff [Tor Myklebust] Fix style error. 4ef0313 [Tor Myklebust] Merge branch 'master' of github.com:apache/spark into alsopt 114fb74 [Tor Myklebust] Turn some 'for' loops into 'while' loops. dcf583a [Tor Myklebust] Remove the partitioner member variable; instead, thread that needle everywhere it needs to go. 23d6f91 [Tor Myklebust] Stop making the partitioner configurable. 495784f [Tor Myklebust] Merge branch 'master' of https://github.com/apache/spark 674933a [Tor Myklebust] Fix style. 40edc23 [Tor Myklebust] Fix missing space. f841345 [Tor Myklebust] Fix daft bug creating 'pairs', also for -> foreach. 5ec9e6c [Tor Myklebust] Clean a couple of things up using 'map'. 36a0f43 [Tor Myklebust] Make the partitioner private. d872b09 [Tor Myklebust] Add negative id ALS test. df27697 [Tor Myklebust] Support custom partitioners. Currently we use the same partitioner for users and products. c90b6d8 [Tor Myklebust] Scramble user and product ids before bucketing. c774d7d [Tor Myklebust] Make the partitioner a member variable and use it instead of modding directly. --- .../spark/mllib/recommendation/ALS.scala | 22 ++++++++++++++----- 1 file changed, 17 insertions(+), 5 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 2a77e1a9efb2e..0cf9a7f909081 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 @@ -472,13 +472,15 @@ class ALS private ( // Compute the XtX and Xy values for each user by adding products it rated in each product // block for (productBlock <- 0 until numBlocks) { - for (p <- 0 until blockFactors(productBlock).length) { + var p = 0 + while (p < blockFactors(productBlock).length) { val x = wrapDoubleArray(blockFactors(productBlock)(p)) tempXtX.fill(0.0) dspr(1.0, x, tempXtX) val (us, rs) = inLinkBlock.ratingsForBlock(productBlock)(p) - for (i <- 0 until us.length) { - if (implicitPrefs) { + if (implicitPrefs) { + var i = 0 + while (i < us.length) { // 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)) @@ -489,11 +491,17 @@ class ALS private ( if (rs(i) > 0) { SimpleBlas.axpy(confidence, x, userXy(us(i))) } - } else { + i += 1 + } + } else { + var i = 0 + while (i < us.length) { userXtX(us(i)).addi(tempXtX) SimpleBlas.axpy(rs(i), x, userXy(us(i))) + i += 1 } } + p += 1 } } @@ -502,7 +510,11 @@ class ALS private ( // Compute the full XtX matrix from the lower-triangular part we got above fillFullMatrix(userXtX(index), fullXtX) // Add regularization - (0 until rank).foreach(i => fullXtX.data(i*rank + i) += lambda) + var i = 0 + while (i < rank) { + fullXtX.data(i * rank + i) += lambda + i += 1 + } // Solve the resulting matrix, which is symmetric and positive-definite if (implicitPrefs) { Solve.solvePositive(fullXtX.addi(YtY.get.value), userXy(index)).data From b3d2ab6b35b31b79ded2a1361f0d4eb17ab78b08 Mon Sep 17 00:00:00 2001 From: Chen Chao Date: Tue, 29 Apr 2014 22:05:40 -0700 Subject: [PATCH 214/641] Args for worker rather than master Args for worker rather than master Author: Chen Chao Closes #587 from CrazyJvm/patch-6 and squashes the following commits: b54b89f [Chen Chao] Args for worker rather than master --- .../scala/org/apache/spark/deploy/worker/WorkerArguments.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 3836bf219ed3e..dc5158102054e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -22,7 +22,7 @@ import java.lang.management.ManagementFactory import org.apache.spark.util.{IntParam, MemoryParam, Utils} /** - * Command-line parser for the master. + * Command-line parser for the worker. */ private[spark] class WorkerArguments(args: Array[String]) { var host = Utils.localHostName() From 7025dda8fa84b57d6f12bc770df2fa10eef21d88 Mon Sep 17 00:00:00 2001 From: WangTao Date: Tue, 29 Apr 2014 22:07:20 -0700 Subject: [PATCH 215/641] Handle the vals that never used In XORShiftRandom.scala, use val "million" instead of constant "1e6.toInt". Delete vals that never used in other files. Author: WangTao Closes #565 from WangTaoTheTonic/master and squashes the following commits: 17cacfc [WangTao] Handle the unused assignment, method parameters and symbol inspected by Intellij IDEA 37b4090 [WangTao] Handle the vals that never used --- .../main/scala/org/apache/spark/network/SecurityMessage.scala | 1 - .../scala/org/apache/spark/partial/GroupedMeanEvaluator.scala | 1 - .../scala/org/apache/spark/storage/BlockFetcherIterator.scala | 2 -- .../scala/org/apache/spark/util/random/XORShiftRandom.scala | 2 +- .../src/main/java/org/apache/spark/examples/JavaLogQuery.java | 1 - examples/src/main/java/org/apache/spark/examples/JavaTC.java | 2 +- .../main/java/org/apache/spark/examples/sql/JavaSparkSQL.java | 1 - 7 files changed, 2 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala b/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala index a1dfc4094cca7..9af9e2e8e9e59 100644 --- a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala @@ -106,7 +106,6 @@ private[spark] class SecurityMessage() extends Logging { * @return BufferMessage */ def toBufferMessage: BufferMessage = { - val startTime = System.currentTimeMillis val buffers = new ArrayBuffer[ByteBuffer]() // 4 bytes for the length of the connectionId diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala index b5111891ed35b..af26c3d59ac02 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala @@ -61,7 +61,6 @@ private[spark] class GroupedMeanEvaluator[T](totalOutputs: Int, confidence: Doub } else if (outputsMerged == 0) { new HashMap[T, BoundedDouble] } else { - val p = outputsMerged.toDouble / totalOutputs val studentTCacher = new StudentTCacher(confidence) val result = new JHashMap[T, BoundedDouble](sums.size) val iter = sums.entrySet.iterator() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index a02dd9441d679..408a797088059 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -118,11 +118,9 @@ object BlockFetcherIterator { }) bytesInFlight += req.size val sizeMap = req.blocks.toMap // so we can look up the size of each blockID - val fetchStart = System.currentTimeMillis() val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) future.onSuccess { case Some(message) => { - val fetchDone = System.currentTimeMillis() val bufferMessage = message.asInstanceOf[BufferMessage] val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) for (blockMessage <- blockMessageArray) { diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index 7f220383f9f8b..55b5713706178 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -91,7 +91,7 @@ private[spark] object XORShiftRandom { val xorRand = new XORShiftRandom(seed) // this is just to warm up the JIT - we're not timing anything - timeIt(1e6.toInt) { + timeIt(million) { javaRand.nextInt() xorRand.nextInt() } 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 2a4278d3c30e5..3f7a879538016 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -75,7 +75,6 @@ public String toString() { public static Tuple3 extractKey(String line) { Matcher m = apacheLogRegex.matcher(line); - List key = Collections.emptyList(); if (m.find()) { String ip = m.group(1); String user = m.group(3); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index 1d776940f06c6..d66b9ba265fe8 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -85,7 +85,7 @@ public Tuple2 call(Tuple2 e) { } }); - long oldCount = 0; + long oldCount; long nextCount = tc.count(); do { oldCount = nextCount; diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index b5b438e9753a7..d62a72f53443c 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -23,7 +23,6 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.sql.api.java.JavaSQLContext; import org.apache.spark.sql.api.java.JavaSchemaRDD; From ff5be9a41e52454e0f9cae83dd1fd50fbeaa684a Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 29 Apr 2014 23:24:34 -0700 Subject: [PATCH 216/641] SPARK-1004. PySpark on YARN This reopens https://github.com/apache/incubator-spark/pull/640 against the new repo Author: Sandy Ryza Closes #30 from sryza/sandy-spark-1004 and squashes the following commits: 89889d4 [Sandy Ryza] Move unzipping py4j to the generate-resources phase so that it gets included in the jar the first time 5165a02 [Sandy Ryza] Fix docs fd0df79 [Sandy Ryza] PySpark on YARN --- bin/pyspark | 1 + bin/pyspark2.cmd | 1 + core/pom.xml | 42 +++++++++++++++++++ .../api/python/PythonWorkerFactory.scala | 10 +---- docs/python-programming-guide.md | 3 ++ python/.gitignore | 3 ++ python/lib/PY4J_VERSION.txt | 1 - python/pyspark/__init__.py | 7 ---- python/pyspark/java_gateway.py | 29 ++++++++++++- python/pyspark/tests.py | 4 +- sbin/spark-config.sh | 3 ++ 11 files changed, 85 insertions(+), 19 deletions(-) delete mode 100644 python/lib/PY4J_VERSION.txt diff --git a/bin/pyspark b/bin/pyspark index cad982bc33477..f5558853e8a4e 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -46,6 +46,7 @@ export PYSPARK_PYTHON # Add the PySpark classes to the Python path: export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH +export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.1-src.zip:$PYTHONPATH # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP=$PYTHONSTARTUP diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index 95791095ec932..d7cfd5eec501c 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -45,6 +45,7 @@ rem Figure out which Python to use. if "x%PYSPARK_PYTHON%"=="x" set PYSPARK_PYTHON=python set PYTHONPATH=%FWDIR%python;%PYTHONPATH% +set PYTHONPATH=%FWDIR%python\lib\py4j-0.8.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%FWDIR%python\pyspark\shell.py diff --git a/core/pom.xml b/core/pom.xml index 73f573a414050..822b5b1dd7cc2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -294,6 +294,48 @@ + + + org.codehaus.mojo + exec-maven-plugin + 1.2.1 + + + generate-resources + + exec + + + + + unzip + ../python + + -o + lib/py4j*.zip + -d + build + + + + + + + src/main/resources + + + ../python + + pyspark/*.py + + + + ../python/build + + py4j/*.py + + + diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index a5f0f3d5e7eae..02799ce0091b0 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -78,12 +78,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String serverSocket = new ServerSocket(0, 1, InetAddress.getByAddress(Array(127, 0, 0, 1))) // Create and start the worker - val sparkHome = new ProcessBuilder().environment().get("SPARK_HOME") - val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/worker.py")) + val pb = new ProcessBuilder(Seq(pythonExec, "-m", "pyspark.worker")) val workerEnv = pb.environment() workerEnv.putAll(envVars) - val pythonPath = sparkHome + "/python/" + File.pathSeparator + workerEnv.get("PYTHONPATH") - workerEnv.put("PYTHONPATH", pythonPath) val worker = pb.start() // Redirect the worker's stderr to ours @@ -154,12 +151,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String try { // Create and start the daemon - val sparkHome = new ProcessBuilder().environment().get("SPARK_HOME") - val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/daemon.py")) + val pb = new ProcessBuilder(Seq(pythonExec, "-m", "pyspark.daemon")) val workerEnv = pb.environment() workerEnv.putAll(envVars) - val pythonPath = sparkHome + "/python/" + File.pathSeparator + workerEnv.get("PYTHONPATH") - workerEnv.put("PYTHONPATH", pythonPath) daemon = pb.start() // Redirect the stderr to ours diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 98233bf556b79..98c456228af9f 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -63,6 +63,9 @@ All of PySpark's library dependencies, including [Py4J](http://py4j.sourceforge. Standalone PySpark applications should be run using the `bin/pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh` or `.cmd`. The script automatically adds the `bin/pyspark` package to the `PYTHONPATH`. +# Running PySpark on YARN + +To run PySpark against a YARN cluster, simply set the MASTER environment variable to "yarn-client". # Interactive Use diff --git a/python/.gitignore b/python/.gitignore index 5c56e638f923a..80b361ffbd51c 100644 --- a/python/.gitignore +++ b/python/.gitignore @@ -1,2 +1,5 @@ *.pyc docs/ +pyspark.egg-info +build/ +dist/ diff --git a/python/lib/PY4J_VERSION.txt b/python/lib/PY4J_VERSION.txt deleted file mode 100644 index 04a0cd52a8d9c..0000000000000 --- a/python/lib/PY4J_VERSION.txt +++ /dev/null @@ -1 +0,0 @@ -b7924aabe9c5e63f0a4d8bbd17019534c7ec014e diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 73fe7378ffa63..07df8697bd1a8 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -49,13 +49,6 @@ Main entry point for accessing data stored in Apache Hive.. """ - - -import sys -import os -sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "python/lib/py4j-0.8.1-src.zip")) - - from pyspark.conf import SparkConf from pyspark.context import SparkContext from pyspark.sql import SQLContext diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 6bb6c877c942d..032d960e40998 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -24,10 +24,11 @@ from py4j.java_gateway import java_import, JavaGateway, GatewayClient -SPARK_HOME = os.environ["SPARK_HOME"] +def launch_gateway(): + SPARK_HOME = os.environ["SPARK_HOME"] + set_env_vars_for_yarn() -def launch_gateway(): # Launch the Py4j gateway using Spark's run command so that we pick up the # proper classpath and settings from spark-env.sh on_windows = platform.system() == "Windows" @@ -70,3 +71,27 @@ def run(self): java_import(gateway.jvm, "org.apache.spark.sql.hive.TestHiveContext") java_import(gateway.jvm, "scala.Tuple2") return gateway + +def set_env_vars_for_yarn(): + # Add the spark jar, which includes the pyspark files, to the python path + env_map = parse_env(os.environ.get("SPARK_YARN_USER_ENV", "")) + if "PYTHONPATH" in env_map: + env_map["PYTHONPATH"] += ":spark.jar" + else: + env_map["PYTHONPATH"] = "spark.jar" + + os.environ["SPARK_YARN_USER_ENV"] = ",".join(k + '=' + v for (k, v) in env_map.items()) + +def parse_env(env_str): + # Turns a comma-separated of env settings into a dict that maps env vars to + # their values. + env = {} + for var_str in env_str.split(","): + parts = var_str.split("=") + if len(parts) == 2: + env[parts[0]] = parts[1] + elif len(var_str) > 0: + print "Invalid entry in SPARK_YARN_USER_ENV: " + var_str + sys.exit(1) + + return env diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 527104587fd31..8cf9d9cf1bd66 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -30,10 +30,12 @@ from pyspark.context import SparkContext from pyspark.files import SparkFiles -from pyspark.java_gateway import SPARK_HOME from pyspark.serializers import read_int +SPARK_HOME = os.environ["SPARK_HOME"] + + class PySparkTestCase(unittest.TestCase): def setUp(self): diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index cd2c7b7b0d496..147b506dd5ca3 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -34,3 +34,6 @@ this="$config_bin/$script" export SPARK_PREFIX=`dirname "$this"`/.. export SPARK_HOME=${SPARK_PREFIX} export SPARK_CONF_DIR="$SPARK_HOME/conf" +# Add the PySpark classes to the PYTHONPATH: +export PYTHONPATH=$SPARK_HOME/python:$PYTHONPATH +export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.1-src.zip:$PYTHONPATH From 55100daa6509bed851f6932845deffa861fef245 Mon Sep 17 00:00:00 2001 From: witgo Date: Wed, 30 Apr 2014 09:49:45 -0700 Subject: [PATCH 217/641] Fix SPARK-1629: Spark should inline use of commons-lang `SystemUtils.IS_... ...OS_WINDOWS` Author: witgo Closes #569 from witgo/SPARK-1629 and squashes the following commits: 31520eb [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1629 fcaafd7 [witgo] merge mastet 49e248e [witgo] Fix SPARK-1629: Spark should inline use of commons-lang `SystemUtils.IS_OS_WINDOWS` --- .../main/scala/org/apache/spark/util/Utils.scala | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 79f314c8dd36c..2c934a4bac5d5 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -31,7 +31,6 @@ import scala.reflect.ClassTag import scala.util.Try import com.google.common.io.Files -import org.apache.commons.lang.SystemUtils import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.json4s._ @@ -50,7 +49,7 @@ private[spark] object Utils extends Logging { val random = new Random() def sparkBin(sparkHome: String, which: String): File = { - val suffix = if (SystemUtils.IS_OS_WINDOWS) ".cmd" else "" + val suffix = if (isWindows) ".cmd" else "" new File(sparkHome + File.separator + "bin", which + suffix) } @@ -614,7 +613,7 @@ private[spark] object Utils extends Logging { */ def isSymlink(file: File): Boolean = { if (file == null) throw new NullPointerException("File must not be null") - if (SystemUtils.IS_OS_WINDOWS) return false + if (isWindows) return false val fileInCanonicalDir = if (file.getParent() == null) { file } else { @@ -1018,7 +1017,7 @@ private[spark] object Utils extends Logging { throw new IOException("Destination must be relative") } var cmdSuffix = "" - val linkCmd = if (SystemUtils.IS_OS_WINDOWS) { + val linkCmd = if (isWindows) { // refer to http://technet.microsoft.com/en-us/library/cc771254.aspx cmdSuffix = " /s /e /k /h /y /i" "cmd /c xcopy " @@ -1062,6 +1061,12 @@ private[spark] object Utils extends Logging { getHadoopFileSystem(new URI(path)) } + /** + * return true if this is Windows. + */ + def isWindows = Option(System.getProperty("os.name")). + map(_.startsWith("Windows")).getOrElse(false) + /** * Indicates whether Spark is currently running unit tests. */ From 98b65593bdcfea54010f8c0fdf2a2b840f18c283 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 1 May 2014 01:15:51 -0700 Subject: [PATCH 218/641] SPARK-1691: Support quoted arguments inside of spark-submit. This is a fairly straightforward fix. The bug was reported by @vanzin and the fix was proposed by @deanwampler and myself. Please take a look! Author: Patrick Wendell Closes #609 from pwendell/quotes and squashes the following commits: 8bed767 [Patrick Wendell] SPARK-1691: Support quoted arguments inside of spark-submit. --- bin/spark-submit | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/bin/spark-submit b/bin/spark-submit index dd0d95d9d4002..49bc26252cadf 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -18,7 +18,7 @@ # export SPARK_HOME="$(cd `dirname $0`/..; pwd)" -ORIG_ARGS=$@ +ORIG_ARGS=("$@") while (($#)); do if [ "$1" = "--deploy-mode" ]; then @@ -39,5 +39,5 @@ if [ ! -z $DRIVER_MEMORY ] && [ ! -z $DEPLOY_MODE ] && [ $DEPLOY_MODE = "client" export SPARK_MEM=$DRIVER_MEMORY fi -$SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit $ORIG_ARGS +$SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" From a43d9c14f2083d6632f410d74db98476e0e4d986 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 1 May 2014 21:32:43 -0700 Subject: [PATCH 219/641] [SQL] SPARK-1661 - Fix regex_serde test The JIRA in question is actually reporting a bug with Shark, but I wanted to make sure Spark SQL did not have similar problems. This fixes a bug in our parsing code that was preventing the test from executing, but it looks like the RegexSerDe is working in Spark SQL. Author: Michael Armbrust Closes #595 from marmbrus/fixRegexSerdeTest and squashes the following commits: a4dc612 [Michael Armbrust] Add files created by hive to gitignore. efa6402 [Michael Armbrust] Fix Hive serde_regex test. --- .gitignore | 5 +++ .../org/apache/spark/sql/hive/HiveQl.scala | 6 ++- ...e_regex-0-60462d14b99bb445b02800e9fb22760e | 22 +++++++++++ ...e_regex-1-dea03bd88cbaabcf438b398e23c139f6 | 0 ..._regex-10-c5b3ec90419a40660e5f83736241c429 | 38 +++++++++++++++++++ ..._regex-11-b00710a6f7a3e5b92c2e17da54cd9ad1 | 0 ...e_regex-2-9d00484beaee46cf72b154a1351aeee9 | 0 ...e_regex-3-817190d8871b70611483cd2abe2e55dc | 0 ...e_regex-4-c3e345183543f40a14d2dd742ebd5346 | 2 + ...e_regex-5-e2d14a76a87a6014c6d2d3501a0c9129 | 2 + ...e_regex-6-f818821654f219d1f4e2482951fae4f1 | 0 ...e_regex-7-4db287576a17c0963219ca63ee0b20e0 | 22 +++++++++++ ...e_regex-8-c429ee76b751e674992f61a29c95af77 | 0 ...e_regex-9-f0e8d394ad18dcbd381792fe9bd8894b | 0 .../execution/HiveCompatibilitySuite.scala | 1 + 15 files changed, 97 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e create mode 100644 sql/hive/src/test/resources/golden/serde_regex-1-dea03bd88cbaabcf438b398e23c139f6 create mode 100644 sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 create mode 100644 sql/hive/src/test/resources/golden/serde_regex-11-b00710a6f7a3e5b92c2e17da54cd9ad1 create mode 100644 sql/hive/src/test/resources/golden/serde_regex-2-9d00484beaee46cf72b154a1351aeee9 create mode 100644 sql/hive/src/test/resources/golden/serde_regex-3-817190d8871b70611483cd2abe2e55dc create mode 100644 sql/hive/src/test/resources/golden/serde_regex-4-c3e345183543f40a14d2dd742ebd5346 create mode 100644 sql/hive/src/test/resources/golden/serde_regex-5-e2d14a76a87a6014c6d2d3501a0c9129 create mode 100644 sql/hive/src/test/resources/golden/serde_regex-6-f818821654f219d1f4e2482951fae4f1 create mode 100644 sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 create mode 100644 sql/hive/src/test/resources/golden/serde_regex-8-c429ee76b751e674992f61a29c95af77 create mode 100644 sql/hive/src/test/resources/golden/serde_regex-9-f0e8d394ad18dcbd381792fe9bd8894b diff --git a/.gitignore b/.gitignore index 857e9feb953bd..a204456d42085 100644 --- a/.gitignore +++ b/.gitignore @@ -49,3 +49,8 @@ unit-tests.log /lib/ rat-results.txt scalastyle.txt + +# For Hive +metastore_db/ +metastore/ +warehouse/ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 1777e96b6713d..1f688fe1117fe 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -347,7 +347,11 @@ private[hive] object HiveQl { protected def nodeToPlan(node: Node): LogicalPlan = node match { // Just fake explain for any of the native commands. case Token("TOK_EXPLAIN", explainArgs) if nativeCommands contains explainArgs.head.getText => - NoRelation + ExplainCommand(NoRelation) + // Create tables aren't native commands due to CTAS queries, but we still don't need to + // explain them. + case Token("TOK_EXPLAIN", explainArgs) if explainArgs.head.getText == "TOK_CREATETABLE" => + ExplainCommand(NoRelation) case Token("TOK_EXPLAIN", explainArgs) => // Ignore FORMATTED if present. val Some(query) :: _ :: _ :: Nil = diff --git a/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e b/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e new file mode 100644 index 0000000000000..d00ee7786a57c --- /dev/null +++ b/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e @@ -0,0 +1,22 @@ +ABSTRACT SYNTAX TREE: + (TOK_CREATETABLE (TOK_TABNAME serde_regex) TOK_LIKETABLE (TOK_TABCOLLIST (TOK_TABCOL host TOK_STRING) (TOK_TABCOL identity TOK_STRING) (TOK_TABCOL user TOK_STRING) (TOK_TABCOL time TOK_STRING) (TOK_TABCOL request TOK_STRING) (TOK_TABCOL status TOK_STRING) (TOK_TABCOL size TOK_INT) (TOK_TABCOL referer TOK_STRING) (TOK_TABCOL agent TOK_STRING)) (TOK_TABLESERIALIZER (TOK_SERDENAME 'org.apache.hadoop.hive.serde2.RegexSerDe' (TOK_TABLEPROPERTIES (TOK_TABLEPROPLIST (TOK_TABLEPROPERTY "input.regex" "([^ ]*) ([^ ]*) ([^ ]*) (-|\\[[^\\]]*\\]) ([^ \"]*|\"[^\"]*\") (-|[0-9]*) (-|[0-9]*)(?: ([^ \"]*|\"[^\"]*\") ([^ \"]*|\"[^\"]*\"))?"))))) TOK_TBLTEXTFILE) + +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Create Table Operator: + Create Table + columns: host string, identity string, user string, time string, request string, status string, size int, referer string, agent string + if not exists: false + input format: org.apache.hadoop.mapred.TextInputFormat + # buckets: -1 + output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat + serde name: org.apache.hadoop.hive.serde2.RegexSerDe + serde properties: + input.regex ([^ ]*) ([^ ]*) ([^ ]*) (-|\[[^\]]*\]) ([^ "]*|"[^"]*") (-|[0-9]*) (-|[0-9]*)(?: ([^ "]*|"[^"]*") ([^ "]*|"[^"]*"))? + name: serde_regex + isExternal: false + + diff --git a/sql/hive/src/test/resources/golden/serde_regex-1-dea03bd88cbaabcf438b398e23c139f6 b/sql/hive/src/test/resources/golden/serde_regex-1-dea03bd88cbaabcf438b398e23c139f6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 new file mode 100644 index 0000000000000..3e290231c27e2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 @@ -0,0 +1,38 @@ +NULL 0 +NULL 0 +-1234567890.123456789 -1234567890 +-4400 4400 +-1255.49 -1255 +-1.122 -11 +-1.12 -1 +-1.12 -1 +-0.333 0 +-0.33 0 +-0.3 0 +0 0 +0 0 +0.01 0 +0.02 0 +0.1 0 +0.2 0 +0.3 0 +0.33 0 +0.333 0 +0.9999999999999999999999999 1 +1 1 +1 1 +1.12 1 +1.122 1 +2 2 +2 2 +3.14 3 +3.14 3 +3.14 3 +3.14 4 +10 10 +20 20 +100 100 +124 124 +125.2 125 +200 200 +1234567890.12345678 1234567890 diff --git a/sql/hive/src/test/resources/golden/serde_regex-11-b00710a6f7a3e5b92c2e17da54cd9ad1 b/sql/hive/src/test/resources/golden/serde_regex-11-b00710a6f7a3e5b92c2e17da54cd9ad1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/serde_regex-2-9d00484beaee46cf72b154a1351aeee9 b/sql/hive/src/test/resources/golden/serde_regex-2-9d00484beaee46cf72b154a1351aeee9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/serde_regex-3-817190d8871b70611483cd2abe2e55dc b/sql/hive/src/test/resources/golden/serde_regex-3-817190d8871b70611483cd2abe2e55dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/serde_regex-4-c3e345183543f40a14d2dd742ebd5346 b/sql/hive/src/test/resources/golden/serde_regex-4-c3e345183543f40a14d2dd742ebd5346 new file mode 100644 index 0000000000000..c55f3dd475574 --- /dev/null +++ b/sql/hive/src/test/resources/golden/serde_regex-4-c3e345183543f40a14d2dd742ebd5346 @@ -0,0 +1,2 @@ +127.0.0.1 - frank [10/Oct/2000:13:55:36 -0700] "GET /apache_pb.gif HTTP/1.0" 200 2326 NULL NULL +127.0.0.1 - - [26/May/2009:00:00:00 +0000] "GET /someurl/?track=Blabla(Main) HTTP/1.1" 200 5864 - "Mozilla/5.0 (Windows; U; Windows NT 6.0; en-US) AppleWebKit/525.19 (KHTML, like Gecko) Chrome/1.0.154.65 Safari/525.19" diff --git a/sql/hive/src/test/resources/golden/serde_regex-5-e2d14a76a87a6014c6d2d3501a0c9129 b/sql/hive/src/test/resources/golden/serde_regex-5-e2d14a76a87a6014c6d2d3501a0c9129 new file mode 100644 index 0000000000000..8bd185bc66ebb --- /dev/null +++ b/sql/hive/src/test/resources/golden/serde_regex-5-e2d14a76a87a6014c6d2d3501a0c9129 @@ -0,0 +1,2 @@ +127.0.0.1 2326 200 [10/Oct/2000:13:55:36 -0700] +127.0.0.1 5864 200 [26/May/2009:00:00:00 +0000] diff --git a/sql/hive/src/test/resources/golden/serde_regex-6-f818821654f219d1f4e2482951fae4f1 b/sql/hive/src/test/resources/golden/serde_regex-6-f818821654f219d1f4e2482951fae4f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 b/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 new file mode 100644 index 0000000000000..da61769c6599d --- /dev/null +++ b/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 @@ -0,0 +1,22 @@ +ABSTRACT SYNTAX TREE: + (TOK_CREATETABLE (TOK_TABNAME serde_regex1) TOK_LIKETABLE (TOK_TABCOLLIST (TOK_TABCOL key TOK_DECIMAL) (TOK_TABCOL value TOK_INT)) (TOK_TABLESERIALIZER (TOK_SERDENAME 'org.apache.hadoop.hive.serde2.RegexSerDe' (TOK_TABLEPROPERTIES (TOK_TABLEPROPLIST (TOK_TABLEPROPERTY "input.regex" "([^ ]*) ([^ ]*)"))))) TOK_TBLTEXTFILE) + +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Create Table Operator: + Create Table + columns: key decimal, value int + if not exists: false + input format: org.apache.hadoop.mapred.TextInputFormat + # buckets: -1 + output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat + serde name: org.apache.hadoop.hive.serde2.RegexSerDe + serde properties: + input.regex ([^ ]*) ([^ ]*) + name: serde_regex1 + isExternal: false + + diff --git a/sql/hive/src/test/resources/golden/serde_regex-8-c429ee76b751e674992f61a29c95af77 b/sql/hive/src/test/resources/golden/serde_regex-8-c429ee76b751e674992f61a29c95af77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/serde_regex-9-f0e8d394ad18dcbd381792fe9bd8894b b/sql/hive/src/test/resources/golden/serde_regex-9-f0e8d394ad18dcbd381792fe9bd8894b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 0bb76f31c373d..6c08e63a39492 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -568,6 +568,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "select_unquote_and", "select_unquote_not", "select_unquote_or", + "serde_regex", "serde_reported_schema", "set_variable_sub", "show_describe_func_quotes", From 55c760ff9bc8079e3df3b63a6ba4e6778a33da47 Mon Sep 17 00:00:00 2001 From: wangfei Date: Thu, 1 May 2014 21:37:22 -0700 Subject: [PATCH 220/641] fix the spelling mistake Author: wangfei Closes #614 from scwf/pxcw and squashes the following commits: d1016ba [wangfei] fix spelling mistake --- core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index dd0a1360abe14..195da81460052 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -244,7 +244,7 @@ private[spark] class Worker( } } catch { case e: Exception => { - logError("Failed to launch exector %s/%d for %s".format(appId, execId, appDesc.name)) + logError("Failed to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) if (executors.contains(appId + "/" + execId)) { executors(appId + "/" + execId).kill() executors -= appId + "/" + execId From 40cf6d31019c5402e5eb08158856242d20697ba4 Mon Sep 17 00:00:00 2001 From: witgo Date: Thu, 1 May 2014 21:39:40 -0700 Subject: [PATCH 221/641] SPARK-1659: improvements spark-submit usage Author: witgo Closes #581 from witgo/SPARK-1659 and squashes the following commits: 0b2cf98 [witgo] Delete spark-submit obsolete usage: "--arg ARG" --- .../scala/org/apache/spark/deploy/SparkSubmitArguments.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 5834dc40f1f95..45defb9a3f92d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -284,8 +284,6 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. | --deploy-mode DEPLOY_MODE Mode to deploy the app in, either 'client' or 'cluster'. | --class CLASS_NAME Name of your app's main class (required for Java apps). - | --arg ARG Argument to be passed to your application's main class. This - | option can be specified multiple times for multiple args. | --name NAME The name of your application (Default: 'Spark'). | --jars JARS A comma-separated list of local jars to include on the | driver classpath and that SparkContext.addJar will work From 394d8cb1c4dfd1e496562009e716b8fc06be22cd Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 1 May 2014 21:42:06 -0700 Subject: [PATCH 222/641] Add tests for FileLogger, EventLoggingListener, and ReplayListenerBus Modifications to Spark core are limited to exposing functionality to test files + minor style fixes. (728 / 769 lines are from tests) Author: Andrew Or Closes #591 from andrewor14/event-log-tests and squashes the following commits: 2883837 [Andrew Or] Merge branch 'master' of github.com:apache/spark into event-log-tests c3afcea [Andrew Or] Compromise 2d5daf8 [Andrew Or] Use temp directory provided by the OS rather than /tmp 2b52151 [Andrew Or] Remove unnecessary file delete + add a comment 62010fd [Andrew Or] More cleanup (renaming variables, updating comments etc) ad2beff [Andrew Or] Clean up EventLoggingListenerSuite + modify a few comments 862e752 [Andrew Or] Merge branch 'master' of github.com:apache/spark into event-log-tests e0ba2f8 [Andrew Or] Fix test failures caused by race condition in processing/mutating events b990453 [Andrew Or] ReplayListenerBus suite - tests do not all pass yet ab66a84 [Andrew Or] Tests for FileLogger + delete file after tests 187bb25 [Andrew Or] Formatting and renaming variables 769336f [Andrew Or] Merge branch 'master' of github.com:apache/spark into event-log-tests 5d38ffe [Andrew Or] Clean up EventLoggingListenerSuite + add comments e12f4b1 [Andrew Or] Preliminary tests for EventLoggingListener (need major cleanup) --- .../scheduler/EventLoggingListener.scala | 40 +- .../spark/scheduler/SparkListenerBus.scala | 2 +- .../org/apache/spark/util/FileLogger.scala | 28 +- .../org/apache/spark/util/JsonProtocol.scala | 10 +- .../scala/org/apache/spark/util/Utils.scala | 18 +- .../scheduler/EventLoggingListenerSuite.scala | 400 ++++++++++++++++++ .../spark/scheduler/ReplayListenerSuite.scala | 166 ++++++++ .../apache/spark/util/FileLoggerSuite.scala | 163 +++++++ 8 files changed, 791 insertions(+), 36 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index d822a8e55111a..7968a0691db10 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -18,13 +18,16 @@ package org.apache.spark.scheduler import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.fs.permission.FsPermission +import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{FileLogger, JsonProtocol} @@ -40,31 +43,36 @@ import org.apache.spark.util.{FileLogger, JsonProtocol} */ private[spark] class EventLoggingListener( appName: String, - conf: SparkConf, - hadoopConfiguration: Configuration) + sparkConf: SparkConf, + hadoopConf: Configuration = SparkHadoopUtil.get.newConfiguration()) extends SparkListener with Logging { import EventLoggingListener._ - private val shouldCompress = conf.getBoolean("spark.eventLog.compress", false) - private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", false) - private val outputBufferSize = conf.getInt("spark.eventLog.buffer.kb", 100) * 1024 - private val logBaseDir = conf.get("spark.eventLog.dir", "/tmp/spark-events").stripSuffix("/") + private val shouldCompress = sparkConf.getBoolean("spark.eventLog.compress", false) + private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false) + private val testing = sparkConf.getBoolean("spark.eventLog.testing", false) + private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 + private val logBaseDir = sparkConf.get("spark.eventLog.dir", DEFAULT_LOG_DIR).stripSuffix("/") private val name = appName.replaceAll("[ :/]", "-").toLowerCase + "-" + System.currentTimeMillis val logDir = logBaseDir + "/" + name - private val logger = - new FileLogger(logDir, conf, hadoopConfiguration, outputBufferSize, shouldCompress, - shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) + protected val logger = new FileLogger(logDir, sparkConf, hadoopConf, outputBufferSize, + shouldCompress, shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) + + // For testing. Keep track of all JSON serialized events that have been logged. + private[scheduler] val loggedEvents = new ArrayBuffer[JValue] /** * Begin logging events. * If compression is used, log a file that indicates which compression library is used. */ def start() { + logger.start() logInfo("Logging events to %s".format(logDir)) if (shouldCompress) { - val codec = conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC) + val codec = + sparkConf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC) logger.newFile(COMPRESSION_CODEC_PREFIX + codec) } logger.newFile(SPARK_VERSION_PREFIX + SparkContext.SPARK_VERSION) @@ -73,11 +81,14 @@ private[spark] class EventLoggingListener( /** Log the event as JSON. */ private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { - val eventJson = compact(render(JsonProtocol.sparkEventToJson(event))) - logger.logLine(eventJson) + val eventJson = JsonProtocol.sparkEventToJson(event) + logger.logLine(compact(render(eventJson))) if (flushLogger) { logger.flush() } + if (testing) { + loggedEvents += eventJson + } } // Events that do not trigger a flush @@ -121,13 +132,12 @@ private[spark] class EventLoggingListener( } private[spark] object EventLoggingListener extends Logging { + val DEFAULT_LOG_DIR = "/tmp/spark-events" val LOG_PREFIX = "EVENT_LOG_" val SPARK_VERSION_PREFIX = "SPARK_VERSION_" val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" - val LOG_FILE_PERMISSIONS: FsPermission = - FsPermission.createImmutable(Integer.parseInt("770", 8).toShort) - + val LOG_FILE_PERMISSIONS = FsPermission.createImmutable(Integer.parseInt("770", 8).toShort) // A cache for compression codecs to avoid creating the same codec many times private val codecMap = new mutable.HashMap[String, CompressionCodec] diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index d6df193d9bcf8..0286aac8769b2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -37,7 +37,7 @@ private[spark] trait SparkListenerBus { * Post an event to all attached listeners. This does nothing if the event is * SparkListenerShutdown. */ - protected def postToAll(event: SparkListenerEvent) { + def postToAll(event: SparkListenerEvent) { event match { case stageSubmitted: SparkListenerStageSubmitted => sparkListeners.foreach(_.onStageSubmitted(stageSubmitted)) diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 0965e0f0f7828..0e6d21b22023a 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import java.io.{FileOutputStream, BufferedOutputStream, PrintWriter, IOException} +import java.io.{BufferedOutputStream, FileOutputStream, IOException, PrintWriter} import java.net.URI import java.text.SimpleDateFormat import java.util.Date @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} import org.apache.hadoop.fs.permission.FsPermission import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec /** @@ -39,8 +40,8 @@ import org.apache.spark.io.CompressionCodec */ private[spark] class FileLogger( logDir: String, - conf: SparkConf, - hadoopConfiguration: Configuration, + sparkConf: SparkConf, + hadoopConf: Configuration = SparkHadoopUtil.get.newConfiguration(), outputBufferSize: Int = 8 * 1024, // 8 KB compress: Boolean = false, overwrite: Boolean = true, @@ -55,14 +56,19 @@ private[spark] class FileLogger( var fileIndex = 0 // Only used if compression is enabled - private lazy val compressionCodec = CompressionCodec.createCodec(conf) + private lazy val compressionCodec = CompressionCodec.createCodec(sparkConf) // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None private var writer: Option[PrintWriter] = None - createLogDir() + /** + * Start this logger by creating the logging directory. + */ + def start() { + createLogDir() + } /** * Create a logging directory with the given path. @@ -83,7 +89,7 @@ private[spark] class FileLogger( } if (dirPermissions.isDefined) { val fsStatus = fileSystem.getFileStatus(path) - if (fsStatus.getPermission().toShort() != dirPermissions.get.toShort) { + if (fsStatus.getPermission.toShort != dirPermissions.get.toShort) { fileSystem.setPermission(path, dirPermissions.get) } } @@ -92,14 +98,14 @@ private[spark] class FileLogger( /** * Create a new writer for the file identified by the given path. * If the permissions are not passed in, it will default to use the permissions - * (dirpermissions) used when class was instantiated. + * (dirPermissions) used when class was instantiated. */ private def createWriter(fileName: String, perms: Option[FsPermission] = None): PrintWriter = { val logPath = logDir + "/" + fileName val uri = new URI(logPath) - val defaultFs = FileSystem.getDefaultUri(hadoopConfiguration).getScheme - val isDefaultLocal = (defaultFs == null || defaultFs == "file") val path = new Path(logPath) + val defaultFs = FileSystem.getDefaultUri(hadoopConf).getScheme + val isDefaultLocal = defaultFs == null || defaultFs == "file" /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). * Therefore, for local files, use FileOutputStream instead. */ @@ -112,7 +118,7 @@ private[spark] class FileLogger( hadoopDataStream.get } - perms.orElse(dirPermissions).foreach {p => fileSystem.setPermission(path, p)} + perms.orElse(dirPermissions).foreach { p => fileSystem.setPermission(path, p) } val bstream = new BufferedOutputStream(dstream, outputBufferSize) val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream new PrintWriter(cstream) @@ -127,7 +133,7 @@ private[spark] class FileLogger( val writeInfo = if (!withTime) { msg } else { - val date = new Date(System.currentTimeMillis()) + val date = new Date(System.currentTimeMillis) dateFormat.get.format(date) + ": " + msg } writer.foreach(_.print(writeInfo)) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 9aed3e0985654..09825087bb048 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -646,11 +646,11 @@ private[spark] object JsonProtocol { } def propertiesFromJson(json: JValue): Properties = { - val properties = new Properties() - if (json != JNothing) { - mapFromJson(json).map { case (k, v) => properties.setProperty(k, v) } - } - properties + Utils.jsonOption(json).map { value => + val properties = new Properties + mapFromJson(json).foreach { case (k, v) => properties.setProperty(k, v) } + properties + }.getOrElse(null) } def UUIDFromJson(json: JValue): UUID = { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 2c934a4bac5d5..536a740140c03 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1062,15 +1062,25 @@ private[spark] object Utils extends Logging { } /** - * return true if this is Windows. + * Return the absolute path of a file in the given directory. */ - def isWindows = Option(System.getProperty("os.name")). - map(_.startsWith("Windows")).getOrElse(false) + def getFilePath(dir: File, fileName: String): Path = { + assert(dir.isDirectory) + val path = new File(dir, fileName).getAbsolutePath + new Path(path) + } + + /** + * Return true if this is Windows. + */ + def isWindows = { + Option(System.getProperty("os.name")).exists(_.startsWith("Windows")) + } /** * Indicates whether Spark is currently running unit tests. */ - private[spark] def isTesting = { + def isTesting = { sys.env.contains("SPARK_TESTING") || sys.props.contains("spark.testing") } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala new file mode 100644 index 0000000000000..95f5bcd855665 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -0,0 +1,400 @@ +/* + * 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.scheduler + +import scala.collection.mutable +import scala.io.Source +import scala.util.Try + +import com.google.common.io.Files +import org.apache.hadoop.fs.{FileStatus, Path} +import org.json4s.jackson.JsonMethods._ +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.{JsonProtocol, Utils} + +/** + * Test whether EventLoggingListener logs events properly. + * + * This tests whether EventLoggingListener actually creates special files while logging events, + * whether the parsing of these special files is correct, and whether the logged events can be + * read and deserialized into actual SparkListenerEvents. + */ +class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { + private val fileSystem = Utils.getHadoopFileSystem("/") + private val allCompressionCodecs = Seq[String]( + "org.apache.spark.io.LZFCompressionCodec", + "org.apache.spark.io.SnappyCompressionCodec" + ) + private val testDir = Files.createTempDir() + private val logDirPath = Utils.getFilePath(testDir, "spark-events") + + after { + Try { fileSystem.delete(logDirPath, true) } + } + + test("Parse names of special files") { + testParsingFileName() + } + + test("Verify special files exist") { + testSpecialFilesExist() + } + + test("Verify special files exist with compression") { + allCompressionCodecs.foreach { codec => + testSpecialFilesExist(compressionCodec = Some(codec)) + } + } + + test("Parse event logging info") { + testParsingLogInfo() + } + + test("Parse event logging info with compression") { + allCompressionCodecs.foreach { codec => + testParsingLogInfo(compressionCodec = Some(codec)) + } + } + + test("Basic event logging") { + testEventLogging() + } + + test("Basic event logging with compression") { + allCompressionCodecs.foreach { codec => + testEventLogging(compressionCodec = Some(codec)) + } + } + + test("End-to-end event logging") { + testApplicationEventLogging() + } + + test("End-to-end event logging with compression") { + allCompressionCodecs.foreach { codec => + testApplicationEventLogging(compressionCodec = Some(codec)) + } + } + + + /* ----------------- * + * Actual test logic * + * ----------------- */ + + import EventLoggingListenerSuite._ + + /** + * Test whether names of special files are correctly identified and parsed. + */ + private def testParsingFileName() { + val logPrefix = EventLoggingListener.LOG_PREFIX + val sparkVersionPrefix = EventLoggingListener.SPARK_VERSION_PREFIX + val compressionCodecPrefix = EventLoggingListener.COMPRESSION_CODEC_PREFIX + val applicationComplete = EventLoggingListener.APPLICATION_COMPLETE + assert(EventLoggingListener.isEventLogFile(logPrefix + "0")) + assert(EventLoggingListener.isEventLogFile(logPrefix + "100")) + assert(EventLoggingListener.isEventLogFile(logPrefix + "ANYTHING")) + assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "0.9.1")) + assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "1.0.0")) + assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "ANYTHING")) + assert(EventLoggingListener.isApplicationCompleteFile(applicationComplete)) + allCompressionCodecs.foreach { codec => + assert(EventLoggingListener.isCompressionCodecFile(compressionCodecPrefix + codec)) + } + + // Negatives + assert(!EventLoggingListener.isEventLogFile("The greatest man of all mankind")) + assert(!EventLoggingListener.isSparkVersionFile("Will never falter in the face of death!")) + assert(!EventLoggingListener.isCompressionCodecFile("Unless he chooses to leave behind")) + assert(!EventLoggingListener.isApplicationCompleteFile("The very treasure he calls Macbeth")) + + // Verify that parsing is correct + assert(EventLoggingListener.parseSparkVersion(sparkVersionPrefix + "1.0.0") === "1.0.0") + allCompressionCodecs.foreach { codec => + assert(EventLoggingListener.parseCompressionCodec(compressionCodecPrefix + codec) === codec) + } + } + + /** + * Test whether the special files produced by EventLoggingListener exist. + * + * There should be exactly one event log and one spark version file throughout the entire + * execution. If a compression codec is specified, then the compression codec file should + * also exist. Only after the application has completed does the test expect the application + * completed file to be present. + */ + private def testSpecialFilesExist(compressionCodec: Option[String] = None) { + + def assertFilesExist(logFiles: Array[FileStatus], loggerStopped: Boolean) { + val numCompressionCodecFiles = if (compressionCodec.isDefined) 1 else 0 + val numApplicationCompleteFiles = if (loggerStopped) 1 else 0 + assert(logFiles.size === 2 + numCompressionCodecFiles + numApplicationCompleteFiles) + assert(eventLogsExist(logFiles)) + assert(sparkVersionExists(logFiles)) + assert(compressionCodecExists(logFiles) === compressionCodec.isDefined) + assert(applicationCompleteExists(logFiles) === loggerStopped) + assertSparkVersionIsValid(logFiles) + compressionCodec.foreach { codec => + assertCompressionCodecIsValid(logFiles, codec) + } + } + + // Verify logging directory exists + val conf = getLoggingConf(logDirPath, compressionCodec) + val eventLogger = new EventLoggingListener("test", conf) + eventLogger.start() + val logPath = new Path(eventLogger.logDir) + assert(fileSystem.exists(logPath)) + val logDir = fileSystem.getFileStatus(logPath) + assert(logDir.isDir) + + // Verify special files are as expected before stop() + var logFiles = fileSystem.listStatus(logPath) + assert(logFiles != null) + assertFilesExist(logFiles, loggerStopped = false) + + // Verify special files are as expected after stop() + eventLogger.stop() + logFiles = fileSystem.listStatus(logPath) + assertFilesExist(logFiles, loggerStopped = true) + } + + /** + * Test whether EventLoggingListener correctly parses the correct information from the logs. + * + * This includes whether it returns the correct Spark version, compression codec (if any), + * and the application's completion status. + */ + private def testParsingLogInfo(compressionCodec: Option[String] = None) { + + def assertInfoCorrect(info: EventLoggingInfo, loggerStopped: Boolean) { + assert(info.logPaths.size > 0) + assert(info.sparkVersion === SparkContext.SPARK_VERSION) + assert(info.compressionCodec.isDefined === compressionCodec.isDefined) + info.compressionCodec.foreach { codec => + assert(compressionCodec.isDefined) + val expectedCodec = compressionCodec.get.split('.').last + assert(codec.getClass.getSimpleName === expectedCodec) + } + assert(info.applicationComplete === loggerStopped) + } + + // Verify that all information is correctly parsed before stop() + val conf = getLoggingConf(logDirPath, compressionCodec) + val eventLogger = new EventLoggingListener("test", conf) + eventLogger.start() + var eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) + assertInfoCorrect(eventLoggingInfo, loggerStopped = false) + + // Verify that all information is correctly parsed after stop() + eventLogger.stop() + eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) + assertInfoCorrect(eventLoggingInfo, loggerStopped = true) + } + + /** + * Test basic event logging functionality. + * + * This creates two simple events, posts them to the EventLoggingListener, and verifies that + * exactly these two events are logged in the expected file. + */ + private def testEventLogging(compressionCodec: Option[String] = None) { + val conf = getLoggingConf(logDirPath, compressionCodec) + val eventLogger = new EventLoggingListener("test", conf) + val listenerBus = new LiveListenerBus + val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", 125L, "Mickey") + val applicationEnd = SparkListenerApplicationEnd(1000L) + + // A comprehensive test on JSON de/serialization of all events is in JsonProtocolSuite + eventLogger.start() + listenerBus.start() + listenerBus.addListener(eventLogger) + listenerBus.postToAll(applicationStart) + listenerBus.postToAll(applicationEnd) + + // Verify file contains exactly the two events logged + val eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) + assert(eventLoggingInfo.logPaths.size > 0) + val lines = readFileLines(eventLoggingInfo.logPaths.head, eventLoggingInfo.compressionCodec) + assert(lines.size === 2) + assert(lines(0).contains("SparkListenerApplicationStart")) + assert(lines(1).contains("SparkListenerApplicationEnd")) + assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === applicationStart) + assert(JsonProtocol.sparkEventFromJson(parse(lines(1))) === applicationEnd) + eventLogger.stop() + } + + /** + * Test end-to-end event logging functionality in an application. + * This runs a simple Spark job and asserts that the expected events are logged when expected. + */ + private def testApplicationEventLogging(compressionCodec: Option[String] = None) { + val conf = getLoggingConf(logDirPath, compressionCodec) + val sc = new SparkContext("local", "test", conf) + assert(sc.eventLogger.isDefined) + val eventLogger = sc.eventLogger.get + val expectedLogDir = logDirPath.toString + assert(eventLogger.logDir.startsWith(expectedLogDir)) + + // Begin listening for events that trigger asserts + val eventExistenceListener = new EventExistenceListener(eventLogger) + sc.addSparkListener(eventExistenceListener) + + // Trigger asserts for whether the expected events are actually logged + sc.parallelize(1 to 10000).count() + sc.stop() + + // Ensure all asserts have actually been triggered + eventExistenceListener.assertAllCallbacksInvoked() + } + + /** + * Assert that all of the specified events are logged by the given EventLoggingListener. + */ + private def assertEventsExist(eventLogger: EventLoggingListener, events: Seq[String]) { + val eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) + assert(eventLoggingInfo.logPaths.size > 0) + val lines = readFileLines(eventLoggingInfo.logPaths.head, eventLoggingInfo.compressionCodec) + val eventSet = mutable.Set(events: _*) + lines.foreach { line => + eventSet.foreach { event => + if (line.contains(event)) { + val parsedEvent = JsonProtocol.sparkEventFromJson(parse(line)) + val eventType = Utils.getFormattedClassName(parsedEvent) + if (eventType == event) { + eventSet.remove(event) + } + } + } + } + assert(eventSet.isEmpty, "The following events are missing: " + eventSet.toSeq) + } + + /** + * Read all lines from the file specified by the given path. + * If a compression codec is specified, use it to read the file. + */ + private def readFileLines( + filePath: Path, + compressionCodec: Option[CompressionCodec]): Seq[String] = { + val fstream = fileSystem.open(filePath) + val cstream = + compressionCodec.map { codec => + codec.compressedInputStream(fstream) + }.getOrElse(fstream) + Source.fromInputStream(cstream).getLines().toSeq + } + + /** + * A listener that asserts certain events are logged by the given EventLoggingListener. + * This is necessary because events are posted asynchronously in a different thread. + */ + private class EventExistenceListener(eventLogger: EventLoggingListener) extends SparkListener { + var jobStarted = false + var jobEnded = false + var appEnded = false + + override def onJobStart(jobStart: SparkListenerJobStart) { + assertEventsExist(eventLogger, Seq[String]( + Utils.getFormattedClassName(SparkListenerApplicationStart), + Utils.getFormattedClassName(SparkListenerBlockManagerAdded), + Utils.getFormattedClassName(SparkListenerEnvironmentUpdate) + )) + jobStarted = true + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd) { + assertEventsExist(eventLogger, Seq[String]( + Utils.getFormattedClassName(SparkListenerJobStart), + Utils.getFormattedClassName(SparkListenerJobEnd), + Utils.getFormattedClassName(SparkListenerStageSubmitted), + Utils.getFormattedClassName(SparkListenerStageCompleted), + Utils.getFormattedClassName(SparkListenerTaskStart), + Utils.getFormattedClassName(SparkListenerTaskEnd) + )) + jobEnded = true + } + + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { + assertEventsExist(eventLogger, Seq[String]( + Utils.getFormattedClassName(SparkListenerApplicationEnd) + )) + appEnded = true + } + + def assertAllCallbacksInvoked() { + assert(jobStarted, "JobStart callback not invoked!") + assert(jobEnded, "JobEnd callback not invoked!") + assert(appEnded, "ApplicationEnd callback not invoked!") + } + } + + + /* -------------------------------------------------------- * + * Helper methods for validating state of the special files * + * -------------------------------------------------------- */ + + private def eventLogsExist(logFiles: Array[FileStatus]): Boolean = { + logFiles.map(_.getPath.getName).exists(EventLoggingListener.isEventLogFile) + } + + private def sparkVersionExists(logFiles: Array[FileStatus]): Boolean = { + logFiles.map(_.getPath.getName).exists(EventLoggingListener.isSparkVersionFile) + } + + private def compressionCodecExists(logFiles: Array[FileStatus]): Boolean = { + logFiles.map(_.getPath.getName).exists(EventLoggingListener.isCompressionCodecFile) + } + + private def applicationCompleteExists(logFiles: Array[FileStatus]): Boolean = { + logFiles.map(_.getPath.getName).exists(EventLoggingListener.isApplicationCompleteFile) + } + + private def assertSparkVersionIsValid(logFiles: Array[FileStatus]) { + val file = logFiles.map(_.getPath.getName).find(EventLoggingListener.isSparkVersionFile) + assert(file.isDefined) + assert(EventLoggingListener.parseSparkVersion(file.get) === SparkContext.SPARK_VERSION) + } + + private def assertCompressionCodecIsValid(logFiles: Array[FileStatus], compressionCodec: String) { + val file = logFiles.map(_.getPath.getName).find(EventLoggingListener.isCompressionCodecFile) + assert(file.isDefined) + assert(EventLoggingListener.parseCompressionCodec(file.get) === compressionCodec) + } + +} + + +object EventLoggingListenerSuite { + + /** Get a SparkConf with event logging enabled. */ + def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None) = { + val conf = new SparkConf + conf.set("spark.eventLog.enabled", "true") + conf.set("spark.eventLog.testing", "true") + conf.set("spark.eventLog.dir", logDir.toString) + compressionCodec.foreach { codec => + conf.set("spark.eventLog.compress", "true") + conf.set("spark.io.compression.codec", codec) + } + conf + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala new file mode 100644 index 0000000000000..d1fe1fc348961 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -0,0 +1,166 @@ +/* + * 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.scheduler + +import java.io.PrintWriter + +import scala.util.Try + +import com.google.common.io.Files +import org.json4s.jackson.JsonMethods._ +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.spark.SparkContext._ +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.{JsonProtocol, Utils} + +/** + * Test whether ReplayListenerBus replays events from logs correctly. + */ +class ReplayListenerSuite extends FunSuite with BeforeAndAfter { + private val fileSystem = Utils.getHadoopFileSystem("/") + private val allCompressionCodecs = Seq[String]( + "org.apache.spark.io.LZFCompressionCodec", + "org.apache.spark.io.SnappyCompressionCodec" + ) + private val testDir = Files.createTempDir() + + after { + Try { fileSystem.delete(Utils.getFilePath(testDir, "events.txt"), true) } + Try { fileSystem.delete(Utils.getFilePath(testDir, "test-replay"), true) } + } + + test("Simple replay") { + testSimpleReplay() + } + + test("Simple replay with compression") { + allCompressionCodecs.foreach { codec => + testSimpleReplay(Some(codec)) + } + } + + // This assumes the correctness of EventLoggingListener + test("End-to-end replay") { + testApplicationReplay() + } + + // This assumes the correctness of EventLoggingListener + test("End-to-end replay with compression") { + allCompressionCodecs.foreach { codec => + testApplicationReplay(Some(codec)) + } + } + + + /* ----------------- * + * Actual test logic * + * ----------------- */ + + /** + * Test simple replaying of events. + */ + private def testSimpleReplay(codecName: Option[String] = None) { + val logFilePath = Utils.getFilePath(testDir, "events.txt") + val codec = codecName.map(getCompressionCodec) + val fstream = fileSystem.create(logFilePath) + val cstream = codec.map(_.compressedOutputStream(fstream)).getOrElse(fstream) + val writer = new PrintWriter(cstream) + val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", 125L, "Mickey") + val applicationEnd = SparkListenerApplicationEnd(1000L) + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) + writer.close() + val replayer = new ReplayListenerBus(Seq(logFilePath), fileSystem, codec) + val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath, codecName) + val eventMonster = new EventMonster(conf) + replayer.addListener(eventMonster) + replayer.replay() + assert(eventMonster.loggedEvents.size === 2) + assert(eventMonster.loggedEvents(0) === JsonProtocol.sparkEventToJson(applicationStart)) + assert(eventMonster.loggedEvents(1) === JsonProtocol.sparkEventToJson(applicationEnd)) + } + + /** + * Test end-to-end replaying of events. + * + * This test runs a few simple jobs with event logging enabled, and compares each emitted + * event to the corresponding event replayed from the event logs. This test makes the + * assumption that the event logging behavior is correct (tested in a separate suite). + */ + private def testApplicationReplay(codecName: Option[String] = None) { + val logDirPath = Utils.getFilePath(testDir, "test-replay") + val conf = EventLoggingListenerSuite.getLoggingConf(logDirPath, codecName) + val sc = new SparkContext("local-cluster[2,1,512]", "Test replay", conf) + + // Run a few jobs + sc.parallelize(1 to 100, 1).count() + sc.parallelize(1 to 100, 2).map(i => (i, i)).count() + sc.parallelize(1 to 100, 3).map(i => (i, i)).groupByKey().count() + sc.parallelize(1 to 100, 4).map(i => (i, i)).groupByKey().persist().count() + sc.stop() + + // Prepare information needed for replay + val codec = codecName.map(getCompressionCodec) + val applications = fileSystem.listStatus(logDirPath) + assert(applications != null && applications.size > 0) + val eventLogDir = applications.sortBy(_.getAccessTime).last + assert(eventLogDir.isDir) + val logFiles = fileSystem.listStatus(eventLogDir.getPath) + assert(logFiles != null && logFiles.size > 0) + val logFile = logFiles.find(_.getPath.getName.startsWith("EVENT_LOG_")) + assert(logFile.isDefined) + val logFilePath = logFile.get.getPath + + // Replay events + val replayer = new ReplayListenerBus(Seq(logFilePath), fileSystem, codec) + val eventMonster = new EventMonster(conf) + replayer.addListener(eventMonster) + replayer.replay() + + // Verify the same events are replayed in the same order + assert(sc.eventLogger.isDefined) + val originalEvents = sc.eventLogger.get.loggedEvents + val replayedEvents = eventMonster.loggedEvents + originalEvents.zip(replayedEvents).foreach { case (e1, e2) => assert(e1 === e2) } + } + + /** + * A simple listener that buffers all the events it receives. + * + * The event buffering functionality must be implemented within EventLoggingListener itself. + * This is because of the following race condition: the event may be mutated between being + * processed by one listener and being processed by another. Thus, in order to establish + * a fair comparison between the original events and the replayed events, both functionalities + * must be implemented within one listener (i.e. the EventLoggingListener). + * + * This child listener inherits only the event buffering functionality, but does not actually + * log the events. + */ + private class EventMonster(conf: SparkConf) extends EventLoggingListener("test", conf) { + logger.close() + } + + private def getCompressionCodec(codecName: String) = { + val conf = new SparkConf + conf.set("spark.io.compression.codec", codecName) + CompressionCodec.createCodec(conf) + } + +} diff --git a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala new file mode 100644 index 0000000000000..f675e1e5b4981 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io.IOException + +import scala.io.Source +import scala.util.Try + +import com.google.common.io.Files +import org.apache.hadoop.fs.Path +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.spark.SparkConf +import org.apache.spark.io.CompressionCodec + +/** + * Test writing files through the FileLogger. + */ +class FileLoggerSuite extends FunSuite with BeforeAndAfter { + private val fileSystem = Utils.getHadoopFileSystem("/") + private val allCompressionCodecs = Seq[String]( + "org.apache.spark.io.LZFCompressionCodec", + "org.apache.spark.io.SnappyCompressionCodec" + ) + private val testDir = Files.createTempDir() + private val logDirPath = Utils.getFilePath(testDir, "test-file-logger") + private val logDirPathString = logDirPath.toString + + after { + Try { fileSystem.delete(logDirPath, true) } + } + + test("Simple logging") { + testSingleFile() + } + + test ("Simple logging with compression") { + allCompressionCodecs.foreach { codec => + testSingleFile(Some(codec)) + } + } + + test("Logging multiple files") { + testMultipleFiles() + } + + test("Logging multiple files with compression") { + allCompressionCodecs.foreach { codec => + testMultipleFiles(Some(codec)) + } + } + + test("Logging when directory already exists") { + // Create the logging directory multiple times + new FileLogger(logDirPathString, new SparkConf, overwrite = true).start() + new FileLogger(logDirPathString, new SparkConf, overwrite = true).start() + new FileLogger(logDirPathString, new SparkConf, overwrite = true).start() + + // If overwrite is not enabled, an exception should be thrown + intercept[IOException] { + new FileLogger(logDirPathString, new SparkConf, overwrite = false).start() + } + } + + + /* ----------------- * + * Actual test logic * + * ----------------- */ + + /** + * Test logging to a single file. + */ + private def testSingleFile(codecName: Option[String] = None) { + val conf = getLoggingConf(codecName) + val codec = codecName.map { c => CompressionCodec.createCodec(conf) } + val logger = + if (codecName.isDefined) { + new FileLogger(logDirPathString, conf, compress = true) + } else { + new FileLogger(logDirPathString, conf) + } + logger.start() + assert(fileSystem.exists(logDirPath)) + assert(fileSystem.getFileStatus(logDirPath).isDir) + assert(fileSystem.listStatus(logDirPath).size === 0) + + logger.newFile() + val files = fileSystem.listStatus(logDirPath) + assert(files.size === 1) + val firstFile = files.head + val firstFilePath = firstFile.getPath + + logger.log("hello") + logger.flush() + assert(readFileContent(firstFilePath, codec) === "hello") + + logger.log(" world") + logger.close() + assert(readFileContent(firstFilePath, codec) === "hello world") + } + + /** + * Test logging to multiple files. + */ + private def testMultipleFiles(codecName: Option[String] = None) { + val conf = getLoggingConf(codecName) + val codec = codecName.map { c => CompressionCodec.createCodec(conf) } + val logger = + if (codecName.isDefined) { + new FileLogger(logDirPathString, conf, compress = true) + } else { + new FileLogger(logDirPathString, conf) + } + logger.start() + logger.newFile("Jean_Valjean") + logger.logLine("Who am I?") + logger.logLine("Destiny?") + logger.newFile("John_Valjohn") + logger.logLine("One") + logger.logLine("Two three...") + logger.newFile("Wolverine") + logger.logLine("There was a time") + logger.logLine("A time when our enemies knew honor.") + logger.close() + assert(readFileContent(new Path(logDirPath, "Jean_Valjean"), codec) === "Who am I?\nDestiny?") + assert(readFileContent(new Path(logDirPath, "John_Valjohn"), codec) === "One\nTwo three...") + assert(readFileContent(new Path(logDirPath, "Wolverine"), codec) === + "There was a time\nA time when our enemies knew honor.") + } + + /** + * Read the content of the file specified by the given path. + * If a compression codec is specified, use it to read the file. + */ + private def readFileContent(logPath: Path, codec: Option[CompressionCodec] = None): String = { + val fstream = fileSystem.open(logPath) + val cstream = codec.map(_.compressedInputStream(fstream)).getOrElse(fstream) + Source.fromInputStream(cstream).getLines().mkString("\n") + } + + private def getLoggingConf(codecName: Option[String]) = { + val conf = new SparkConf + codecName.foreach { c => conf.set("spark.io.compression.codec", c) } + conf + } + +} From f25ebed9f4552bc2c88a96aef06729d9fc2ee5b3 Mon Sep 17 00:00:00 2001 From: witgo Date: Fri, 2 May 2014 12:40:27 -0700 Subject: [PATCH 223/641] SPARK-1695: java8-tests compiler error: package com.google.common.co... ...llections does not exist Author: witgo Closes #611 from witgo/SPARK-1695 and squashes the following commits: d77a887 [witgo] Fix SPARK-1695: java8-tests compiler error: package com.google.common.collections does not exist --- .../src/test/java/org/apache/spark/Java8APISuite.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java index 7eb8b45fc3cf0..feabca6733484 100644 --- a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java +++ b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java @@ -23,7 +23,7 @@ import scala.Tuple2; -import com.google.common.collections.Iterables; +import com.google.common.collect.Iterables; import com.google.common.base.Optional; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; From 4bf24f7897e1c67ca5f96dec05480e571f05ee1d Mon Sep 17 00:00:00 2001 From: wangfei Date: Fri, 2 May 2014 21:34:54 -0700 Subject: [PATCH 224/641] delete no use var Author: wangfei Closes #613 from scwf/masterIndex and squashes the following commits: 1463056 [wangfei] delete no use var: masterIndex --- core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 195da81460052..cd6bd2cd29fc7 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -71,8 +71,6 @@ private[spark] class Worker( // TTL for app folders/data; after TTL expires it will be cleaned up val APP_DATA_RETENTION_SECS = conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) - // Index into masterUrls that we're currently trying to register with. - var masterIndex = 0 val masterLock: Object = new Object() var master: ActorSelection = null From 2b961d88079d7a3f9da63d5175d7b61f6dec762b Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 2 May 2014 21:42:31 -0700 Subject: [PATCH 225/641] SPARK-1492. Update Spark YARN docs to use spark-submit Author: Sandy Ryza Closes #601 from sryza/sandy-spark-1492 and squashes the following commits: 5df1634 [Sandy Ryza] Address additional comments from Patrick. be46d1f [Sandy Ryza] Address feedback from Marcelo and Patrick 867a3ea [Sandy Ryza] SPARK-1492. Update Spark YARN docs to use spark-submit --- docs/cluster-overview.md | 15 ++--- docs/running-on-yarn.md | 117 ++++++++++----------------------------- 2 files changed, 38 insertions(+), 94 deletions(-) diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index b011679fede2d..79b0061e2c5a1 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -86,7 +86,7 @@ the `--help` flag. Here are a few examples of common options: --master local[8] \ my-app.jar -# Run on a Spark cluster +# Run on a Spark standalone cluster ./bin/spark-submit \ --class my.main.ClassName --master spark://mycluster:7077 \ @@ -118,21 +118,22 @@ If you are ever unclear where configuration options are coming from. fine-graine information can be printed by adding the `--verbose` option to `./spark-submit`. ### Advanced Dependency Management -When using `./bin/spark-submit` jars will be automatically transferred to the cluster. For many -users this is sufficient. However, advanced users can add jars by calling `addFile` or `addJar` -on an existing SparkContext. This can be used to distribute JAR files (Java/Scala) or .egg and -.zip libraries (Python) to executors. Spark uses the following URL scheme to allow different +When using `./bin/spark-submit` the app jar along with any jars included with the `--jars` option +will be automatically transferred to the cluster. `--jars` can also be used to distribute .egg and .zip +libraries for Python to executors. Spark uses the following URL scheme to allow different strategies for disseminating jars: - **file:** - Absolute paths and `file:/` URIs are served by the driver's HTTP file server, and - every executor pulls the file from the driver HTTP server + every executor pulls the file from the driver HTTP server. - **hdfs:**, **http:**, **https:**, **ftp:** - these pull down files and JARs from the URI as expected - **local:** - a URI starting with local:/ is expected to exist as a local file on each worker node. This means that no network IO will be incurred, and works well for large files/JARs that are pushed to each worker, or shared via NFS, GlusterFS, etc. Note that JARs and files are copied to the working directory for each SparkContext on the executor nodes. -Over time this can use up a significant amount of space and will need to be cleaned up. +This can use up a significant amount of space over time and will need to be cleaned up. With YARN, cleanup +is handled automatically, and with Spark standalone, automatic cleanup can be configured with the +`spark.worker.cleanup.appDataTtl` property. # Monitoring diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 9765062ec689a..68183ee8b4613 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -5,27 +5,13 @@ title: Launching Spark on YARN Support for running on [YARN (Hadoop NextGen)](http://hadoop.apache.org/docs/r2.0.2-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html) -was added to Spark in version 0.6.0, and improved in 0.7.0 and 0.8.0. - -# Building a YARN-Enabled Assembly JAR - -We need a consolidated Spark JAR (which bundles all the required dependencies) to run Spark jobs on a YARN cluster. -This can be built by setting the Hadoop version and `SPARK_YARN` environment variable, as follows: - - SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly - -The assembled JAR will be something like this: -`./assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly_{{site.SPARK_VERSION}}-hadoop2.0.5.jar`. - -The build process now also supports new YARN versions (2.2.x). See below. +was added to Spark in version 0.6.0, and improved in subsequent releases. # Preparations -- Building a YARN-enabled assembly (see above). -- The assembled jar can be installed into HDFS or used locally. -- Your application code must be packaged into a separate JAR file. - -If you want to test out the YARN deployment mode, you can use the current Spark examples. A `spark-examples_{{site.SCALA_BINARY_VERSION}}-{{site.SPARK_VERSION}}` file can be generated by running `sbt/sbt assembly`. NOTE: since the documentation you're reading is for Spark version {{site.SPARK_VERSION}}, we are assuming here that you have downloaded Spark {{site.SPARK_VERSION}} or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different. +Running Spark-on-YARN requires a binary distribution of Spark which is built with YARN support. +Binary distributions can be downloaded from the Spark project website. +To build Spark yourself, refer to the [building with maven guide](building-with-maven.html). # Configuration @@ -44,86 +30,47 @@ System Properties: * `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. * `spark.yarn.historyServer.address`, the address of the Spark history server (i.e. host.com:18080). The address should not contain a scheme (http://). Defaults to not being set since the history server is an optional service. This address is given to the Yarn ResourceManager when the Spark application finishes to link the application from the ResourceManager UI to the Spark history server UI. +By default, Spark on YARN will use a Spark jar installed locally, but the Spark jar can also be in a world-readable location on HDFS. This allows YARN to cache it on nodes so that it doesn't need to be distributed each time an application runs. To point to a jar on HDFS, export SPARK_JAR=hdfs:///some/path. + # Launching Spark on YARN Ensure that HADOOP_CONF_DIR or YARN_CONF_DIR points to the directory which contains the (client side) configuration files for the Hadoop cluster. -These configs are used to connect to the cluster, write to the dfs, and connect to the YARN ResourceManager. +These configs are used to write to the dfs and connect to the YARN ResourceManager. There are two deploy modes that can be used to launch Spark applications on YARN. In yarn-cluster mode, the Spark driver runs inside an application master process which is managed by YARN on the cluster, and the client can go away after initiating the application. In yarn-client mode, the driver runs in the client process, and the application master is only used for requesting resources from YARN. Unlike in Spark standalone and Mesos mode, in which the master's address is specified in the "master" parameter, in YARN mode the ResourceManager's address is picked up from the Hadoop configuration. Thus, the master parameter is simply "yarn-client" or "yarn-cluster". -The spark-submit script described in the [cluster mode overview](cluster-overview.html) provides the most straightforward way to submit a compiled Spark application to YARN in either deploy mode. For info on the lower-level invocations it uses, read ahead. For running spark-shell against YARN, skip down to the yarn-client section. - -## Launching a Spark application with yarn-cluster mode. - -The command to launch the Spark application on the cluster is as follows: - - SPARK_JAR= ./bin/spark-class org.apache.spark.deploy.yarn.Client \ - --jar \ - --class \ - --arg \ - --num-executors \ - --driver-memory \ - --executor-memory \ - --executor-cores \ - --name \ - --queue \ - --addJars \ - --files \ - --archives - -To pass multiple arguments the "arg" option can be specified multiple times. For example: - - # Build the Spark assembly JAR and the Spark examples JAR - $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly - - # Configure logging - $ cp conf/log4j.properties.template conf/log4j.properties - - # Submit Spark's ApplicationMaster to YARN's ResourceManager, and instruct Spark to run the SparkPi example - $ SPARK_JAR=./assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ - ./bin/spark-class org.apache.spark.deploy.yarn.Client \ - --jar examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ - --class org.apache.spark.examples.SparkPi \ - --arg yarn-cluster \ - --arg 5 \ - --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. - -Because the application is run on a remote machine where the Application Master is running, applications that involve local interaction, such as spark-shell, will not work. - -## Launching a Spark application with yarn-client mode. - -With yarn-client mode, the application will be launched locally, just like running an application or spark-shell on Local / Mesos / Standalone client mode. The launch method is also the same, just make sure to specify the master URL as "yarn-client". You also need to export the env value for SPARK_JAR. +To launch a Spark application in yarn-cluster mode: -Configuration in yarn-client mode: + ./bin/spark-submit --class path.to.your.Class --master yarn-cluster [options] [app options] + +For example: -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. + $ ./bin/spark-submit --class org.apache.spark.examples.SparkPi \ + --master yarn-cluster \ + --num-executors 3 \ + --driver-memory 4g \ + --executor-memory 2g \ + --executor-cores 1 + examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ + yarn-cluster 5 -* `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. -* `SPARK_YARN_DIST_ARCHIVES`, Comma separated list of archives to be distributed with the job. +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. -For example: +To launch a Spark application in yarn-client mode, do the same, but replace "yarn-cluster" with "yarn-client". To run spark-shell: - SPARK_JAR=./assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ - ./bin/run-example org.apache.spark.examples.SparkPi yarn-client + $ MASTER=yarn-client ./bin/spark-shell -or +## Adding additional jars - SPARK_JAR=./assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ - MASTER=yarn-client ./bin/spark-shell +In yarn-cluster mode, the driver runs on a different machine than the client, so SparkContext.addJar won't work out of the box with files that are local to the client. To make files on the client available to SparkContext.addJar, include them with the `--jars` option in the launch command. + $ ./bin/spark-submit --class my.main.Class \ + --master yarn-cluster \ + --jars my-other-jar.jar,my-other-other-jar.jar + my-main-jar.jar + yarn-cluster 5 # Viewing logs @@ -135,13 +82,9 @@ will print out the contents of all log files from all containers from the given When log aggregation isn't turned on, logs are retained locally on each machine under YARN_APP_LOGS_DIR, which is usually configured to /tmp/logs or $HADOOP_HOME/logs/userlogs depending on the Hadoop version and installation. Viewing logs for a container requires going to the host that contains them and looking in this directory. Subdirectories organize log files by application ID and container ID. -# Building Spark for Hadoop/YARN 2.2.x - -See [Building Spark with Maven](building-with-maven.html) for instructions on how to build Spark using Maven. - # Important notes - Before Hadoop 2.2, YARN does not support cores in container resource requests. Thus, when running against an earlier version, the numbers of cores given via command line arguments cannot be passed to YARN. Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. - The local directories used by Spark executors will be the local directories configured for YARN (Hadoop YARN config yarn.nodemanager.local-dirs). If the user specifies spark.local.dir, it will be ignored. - The --files and --archives options support specifying file names with the # similar to Hadoop. For example you can specify: --files localtest.txt#appSees.txt and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name appSees.txt, and your application should use the name as appSees.txt to reference it when running on YARN. -- The --addJars option allows the SparkContext.addJar function to work if you are using it with local files. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files. +- The --jars option allows the SparkContext.addJar function to work if you are using it with local files and running in yarn-cluster mode. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files. From 0a14421765b672305e8f32ded4a9a1f6f7241d8d Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Fri, 2 May 2014 23:55:13 -0700 Subject: [PATCH 226/641] SPARK-1700: Close socket file descriptors on task completion This will ensure that sockets do not build up over the course of a job, and that cancellation successfully cleans up sockets. Tested in standalone mode. More file descriptors spawn than expected (around 1000ish rather than the expected 8ish) but they do not pile up between runs, or as high as before (where they went up to around 5k). Author: Aaron Davidson Closes #623 from aarondav/pyspark2 and squashes the following commits: 0ca13bb [Aaron Davidson] SPARK-1700: Close socket file descriptors on task completion --- .../scala/org/apache/spark/api/python/PythonRDD.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) 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 672c344a56597..61407007087c6 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 @@ -54,7 +54,16 @@ private[spark] class PythonRDD[T: ClassTag]( override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { val startTime = System.currentTimeMillis val env = SparkEnv.get - val worker = env.createPythonWorker(pythonExec, envVars.toMap) + val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap) + + // Ensure worker socket is closed on task completion. Closing sockets is idempotent. + context.addOnCompleteCallback(() => + try { + worker.close() + } catch { + case e: Exception => logWarning("Failed to close worker socket", e) + } + ) @volatile var readerException: Exception = null From 9347565f4188cf1574c6dc49fcde91eb286be955 Mon Sep 17 00:00:00 2001 From: ArcherShao Date: Sat, 3 May 2014 00:17:36 -0700 Subject: [PATCH 227/641] Update SchemaRDD.scala Modify spelling errors Author: ArcherShao Closes #619 from ArcherShao/patch-1 and squashes the following commits: 2957195 [ArcherShao] Update SchemaRDD.scala --- .../src/main/scala/org/apache/spark/sql/SchemaRDD.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index ca6e0a696405a..d7782d6b32819 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -133,7 +133,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Project(exprs, logicalPlan)) /** - * Filters the ouput, only returning those rows where `condition` evaluates to true. + * Filters the output, only returning those rows where `condition` evaluates to true. * * {{{ * schemaRDD.where('a === 'b) @@ -151,9 +151,9 @@ class SchemaRDD( * * @param otherPlan the [[SchemaRDD]] that should be joined with this one. * @param joinType One of `Inner`, `LeftOuter`, `RightOuter`, or `FullOuter`. Defaults to `Inner.` - * @param on An optional condition for the join operation. This is equivilent to the `ON` + * @param on An optional condition for the join operation. This is equivalent to the `ON` * clause in standard SQL. In the case of `Inner` joins, specifying a - * `condition` is equivilent to adding `where` clauses after the `join`. + * `condition` is equivalent to adding `where` clauses after the `join`. * * @group Query */ @@ -195,7 +195,7 @@ class SchemaRDD( /** * Applies a qualifier to the attributes of this relation. Can be used to disambiguate attributes - * with the same name, for example, when peforming self-joins. + * with the same name, for example, when performing self-joins. * * {{{ * val x = schemaRDD.where('a === 1).as('x) From 3d0a02dff3011e8894d98d903cd086bc95e56807 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Sat, 3 May 2014 10:59:05 -0700 Subject: [PATCH 228/641] [WIP] SPARK-1676: Cache Hadoop UGIs by default to prevent FileSystem leak Move the doAs in Executor higher up so that we only have 1 ugi and aren't leaking filesystems. Fix spark on yarn to work when the cluster is running as user "yarn" but the clients are launched as the user and want to read/write to hdfs as the user. Note this hasn't been fully tested yet. Need to test in standalone mode. Putting this up for people to look at and possibly test. I don't have access to a mesos cluster. This is alternative to https://github.com/apache/spark/pull/607 Author: Thomas Graves Closes #621 from tgravescs/SPARK-1676 and squashes the following commits: 244d55a [Thomas Graves] fix line length 44163d4 [Thomas Graves] Rework 9398853 [Thomas Graves] change to have doAs in executor higher up. --- .../apache/spark/deploy/SparkHadoopUtil.scala | 17 +++++-- .../CoarseGrainedExecutorBackend.scala | 44 +++++++++++-------- .../org/apache/spark/executor/Executor.scala | 4 +- .../spark/executor/MesosExecutorBackend.scala | 14 +++--- .../spark/deploy/yarn/ApplicationMaster.scala | 10 ++--- .../spark/deploy/yarn/ExecutorLauncher.scala | 7 ++- .../spark/deploy/yarn/ApplicationMaster.scala | 12 ++--- .../spark/deploy/yarn/ExecutorLauncher.scala | 7 ++- 8 files changed, 69 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 498fcc520ac5e..e2df1b8954124 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -24,25 +24,36 @@ import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation -import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.{Logging, SparkContext, SparkException} import scala.collection.JavaConversions._ /** * Contains util methods to interact with Hadoop from Spark. */ -class SparkHadoopUtil { +class SparkHadoopUtil extends Logging { val conf: Configuration = newConfiguration() UserGroupInformation.setConfiguration(conf) - def runAsUser(user: String)(func: () => Unit) { + /** + * Runs the given function with a Hadoop UserGroupInformation as a thread local variable + * (distributed to child threads), used for authenticating HDFS and YARN calls. + * + * IMPORTANT NOTE: If this function is going to be called repeated in the same process + * you need to look https://issues.apache.org/jira/browse/HDFS-3545 and possibly + * do a FileSystem.closeAllForUGI in order to avoid leaking Filesystems + */ + def runAsSparkUser(func: () => Unit) { + val user = Option(System.getenv("SPARK_USER")).getOrElse(SparkContext.SPARK_UNKNOWN_USER) if (user != SparkContext.SPARK_UNKNOWN_USER) { + logDebug("running as user: " + user) val ugi = UserGroupInformation.createRemoteUser(user) transferCredentials(UserGroupInformation.getCurrentUser(), ugi) ugi.doAs(new PrivilegedExceptionAction[Unit] { def run: Unit = func() }) } else { + logDebug("running as SPARK_UNKNOWN_USER") func() } } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 9ac7365f47f9f..e912ae8a5d3c5 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -22,8 +22,9 @@ import java.nio.ByteBuffer import akka.actor._ import akka.remote._ -import org.apache.spark.{SecurityManager, SparkConf, Logging} +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.TaskState.TaskState +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -94,25 +95,30 @@ private[spark] class CoarseGrainedExecutorBackend( private[spark] object CoarseGrainedExecutorBackend { def run(driverUrl: String, executorId: String, hostname: String, cores: Int, - workerUrl: Option[String]) { - // Debug code - Utils.checkHost(hostname) - - val conf = new SparkConf - // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor - // before getting started with all our system properties, etc - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, - indestructible = true, conf = conf, new SecurityManager(conf)) - // set it - val sparkHostPort = hostname + ":" + boundPort - actorSystem.actorOf( - Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, - sparkHostPort, cores), - name = "Executor") - workerUrl.foreach{ url => - actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") + workerUrl: Option[String]) { + + SparkHadoopUtil.get.runAsSparkUser { () => + // Debug code + Utils.checkHost(hostname) + + val conf = new SparkConf + // Create a new ActorSystem to run the backend, because we can't create a + // SparkEnv / Executor before getting started with all our system properties, etc + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, + indestructible = true, conf = conf, new SecurityManager(conf)) + // set it + val sparkHostPort = hostname + ":" + boundPort + actorSystem.actorOf( + Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, + sparkHostPort, cores), + name = "Executor") + workerUrl.foreach { + url => + actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") + } + actorSystem.awaitTermination() + } - actorSystem.awaitTermination() } def main(args: Array[String]) { 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 272bcda5f8f2f..98e7e0be813be 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -128,8 +128,6 @@ private[spark] class Executor( // Maintains the list of running tasks. private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] - val sparkUser = Option(System.getenv("SPARK_USER")).getOrElse(SparkContext.SPARK_UNKNOWN_USER) - def launchTask(context: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) { val tr = new TaskRunner(context, taskId, serializedTask) runningTasks.put(taskId, tr) @@ -172,7 +170,7 @@ private[spark] class Executor( } } - override def run(): Unit = SparkHadoopUtil.get.runAsUser(sparkUser) { () => + override def run() { val startTime = System.currentTimeMillis() SparkEnv.set(env) Thread.currentThread.setContextClassLoader(replClassLoader) diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 64e24506e8038..9b56f711e0e0b 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -23,10 +23,10 @@ import com.google.protobuf.ByteString import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver, MesosNativeLibrary} import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} -import org.apache.spark.Logging -import org.apache.spark.TaskState +import org.apache.spark.{Logging, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.util.Utils +import org.apache.spark.deploy.SparkHadoopUtil private[spark] class MesosExecutorBackend extends MesosExecutor @@ -95,9 +95,11 @@ private[spark] class MesosExecutorBackend */ private[spark] object MesosExecutorBackend { def main(args: Array[String]) { - MesosNativeLibrary.load() - // Create a new Executor and start it running - val runner = new MesosExecutorBackend() - new MesosExecutorDriver(runner).run() + SparkHadoopUtil.get.runAsSparkUser { () => + MesosNativeLibrary.load() + // Create a new Executor and start it running + val runner = new MesosExecutorBackend() + new MesosExecutorDriver(runner).run() + } } } 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 fc13dbecb4555..8f0ecb855718e 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 @@ -70,9 +70,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private var registered = false - private val sparkUser = Option(System.getenv("SPARK_USER")).getOrElse( - SparkContext.SPARK_UNKNOWN_USER) - def run() { // Setup the directories so things go to yarn approved directories rather // then user specified and /tmp. @@ -192,7 +189,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, false /* initialize */ , Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) val t = new Thread { - override def run(): Unit = SparkHadoopUtil.get.runAsUser(sparkUser) { () => + override def run() { + var successed = false try { // Copy @@ -480,6 +478,8 @@ object ApplicationMaster { def main(argStrings: Array[String]) { val args = new ApplicationMasterArguments(argStrings) - new ApplicationMaster(args).run() + SparkHadoopUtil.get.runAsSparkUser { () => + new ApplicationMaster(args).run() + } } } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index 65b7215afbd4c..a3bd91590fc25 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -29,10 +29,11 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import akka.actor._ import akka.remote._ import akka.actor.Terminated -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.SplitInfo +import org.apache.spark.deploy.SparkHadoopUtil /** * An application master that allocates executors on behalf of a driver that is running outside @@ -279,6 +280,8 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp object ExecutorLauncher { def main(argStrings: Array[String]) { val args = new ApplicationMasterArguments(argStrings) - new ExecutorLauncher(args).run() + SparkHadoopUtil.get.runAsSparkUser { () => + new ExecutorLauncher(args).run() + } } } 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 90e807160d4b6..c1dfe3f53b40b 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 @@ -71,9 +71,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) private var registered = false - - private val sparkUser = Option(System.getenv("SPARK_USER")).getOrElse( - SparkContext.SPARK_UNKNOWN_USER) def run() { // Setup the directories so things go to YARN approved directories rather @@ -179,8 +176,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, false /* initialize */ , Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) val t = new Thread { - override def run(): Unit = SparkHadoopUtil.get.runAsUser(sparkUser) { () => - var successed = false + override def run() { + + var successed = false try { // Copy var mainArgs: Array[String] = new Array[String](args.userArgs.size) @@ -462,6 +460,8 @@ object ApplicationMaster { def main(argStrings: Array[String]) { val args = new ApplicationMasterArguments(argStrings) - new ApplicationMaster(args).run() + SparkHadoopUtil.get.runAsSparkUser { () => + new ApplicationMaster(args).run() + } } } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index a14bb377aa133..a4ce8766d347c 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -28,12 +28,13 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import akka.actor._ import akka.remote._ import akka.actor.Terminated -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest +import org.apache.spark.deploy.SparkHadoopUtil /** * An application master that allocates executors on behalf of a driver that is running outside @@ -255,6 +256,8 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp object ExecutorLauncher { def main(argStrings: Array[String]) { val args = new ApplicationMasterArguments(argStrings) - new ExecutorLauncher(args).run() + SparkHadoopUtil.get.runAsSparkUser { () => + new ExecutorLauncher(args).run() + } } } From 11d54941760f86706e28f7ace8ece664c9164ba6 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 3 May 2014 12:31:31 -0700 Subject: [PATCH 229/641] SPARK-1663. Corrections for several compile errors in streaming code examples, and updates to follow API changes I gave the Streaming code examples, both Scala and Java, a test run today. I turned up a number of small errors, mostly compile errors in the Java examples. There were a few typos in the Scala too. I also took the liberty of adding things like imports, since in several cases they are not obvious. Feel free to push back on some changes. There's one thing I haven't quite addressed in the changes. `JavaPairDStream` uses the Java API version of `Function2` in almost all cases, as `JFunction2`. However it uses `scala.Function2` in: ``` def reduceByKeyAndWindow(reduceFunc: Function2[V, V, V], windowDuration: Duration) :JavaPairDStream[K, V] = { dstream.reduceByKeyAndWindow(reduceFunc, windowDuration) } ``` Is that a typo? Also, in Scala, I could not get this to compile: ``` val windowedWordCounts = pairs.reduceByKeyAndWindow(_ + _, Seconds(30), Seconds(10)) error: missing parameter type for expanded function ((x$1, x$2) => x$1.$plus(x$2)) ``` You can see my fix below but am I missing something? Otherwise I can say these all worked for me! Author: Sean Owen Closes #589 from srowen/SPARK-1663 and squashes the following commits: 65a906b [Sean Owen] Corrections for several compile errors in streaming code examples, and updates to follow API changes --- docs/streaming-programming-guide.md | 62 +++++++++++++++++------------ 1 file changed, 36 insertions(+), 26 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 7ad06427cac5c..b22bb45828b20 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -76,16 +76,19 @@ Besides Spark's configuration, we specify that any DStream will be processed in 1 second batches. {% highlight scala %} -// Create a StreamingContext with a SparkConf configuration -val ssc = new StreamingContext(sparkConf, Seconds(1)) +import org.apache.spark.api.java.function._ +import org.apache.spark.streaming._ +import org.apache.spark.streaming.api._ +// Create a StreamingContext with a local master +val ssc = new StreamingContext("local", "NetworkWordCount", Seconds(1)) {% endhighlight %} Using this context, we then create a new DStream by specifying the IP address and port of the data server. {% highlight scala %} -// Create a DStream that will connect to serverIP:serverPort -val lines = ssc.socketTextStream(serverIP, serverPort) +// Create a DStream that will connect to serverIP:serverPort, like localhost:9999 +val lines = ssc.socketTextStream("localhost", 9999) {% endhighlight %} This `lines` DStream represents the stream of data that will be received from the data @@ -103,6 +106,7 @@ each line will be split into multiple words and the stream of words is represent `words` DStream. Next, we want to count these words. {% highlight scala %} +import org.apache.spark.streaming.StreamingContext._ // Count each word in each batch val pairs = words.map(word => (word, 1)) val wordCounts = pairs.reduceByKey(_ + _) @@ -138,16 +142,20 @@ functionality. Besides Spark's configuration, we specify that any DStream would in 1 second batches. {% highlight java %} -// Create a StreamingContext with a SparkConf configuration -JavaStreamingContext jssc = StreamingContext(sparkConf, new Duration(1000)) +import org.apache.spark.api.java.function.*; +import org.apache.spark.streaming.*; +import org.apache.spark.streaming.api.java.*; +import scala.Tuple2; +// Create a StreamingContext with a local master +JavaStreamingContext jssc = new JavaStreamingContext("local", "JavaNetworkWordCount", new Duration(1000)) {% endhighlight %} Using this context, we then create a new DStream by specifying the IP address and port of the data server. {% highlight java %} -// Create a DStream that will connect to serverIP:serverPort -JavaDStream lines = jssc.socketTextStream(serverIP, serverPort); +// Create a DStream that will connect to serverIP:serverPort, like localhost:9999 +JavaDStream lines = jssc.socketTextStream("localhost", 9999); {% endhighlight %} This `lines` DStream represents the stream of data that will be received from the data @@ -159,7 +167,7 @@ space into words. JavaDStream words = lines.flatMap( new FlatMapFunction() { @Override public Iterable call(String x) { - return Lists.newArrayList(x.split(" ")); + return Arrays.asList(x.split(" ")); } }); {% endhighlight %} @@ -359,7 +367,7 @@ as explained earlier. Finally, the last two parameters are needed to deploy your if running in distributed mode, as described in the [Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster). Additionally, the underlying SparkContext can be accessed as -`streamingContext.sparkContext`. +`ssc.sparkContext`. The batch interval must be set based on the latency requirements of your application and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-size) @@ -399,7 +407,7 @@ These operations are discussed in detail in later sections. ## Input Sources -We have already taken a look at the `streamingContext.socketTextStream(...)` in the [quick +We have already taken a look at the `ssc.socketTextStream(...)` in the [quick example](#a-quick-example) which creates a DStream from text data received over a TCP socket connection. Besides sockets, the core Spark Streaming API provides methods for creating DStreams from files and Akka actors as input sources. @@ -409,12 +417,12 @@ Specifically, for files, the DStream can be created as
      {% highlight scala %} -streamingContext.fileStream(dataDirectory) +ssc.fileStream(dataDirectory) {% endhighlight %}
      {% highlight java %} -javaStreamingContext.fileStream(dataDirectory); +jssc.fileStream(dataDirectory); {% endhighlight %}
      @@ -443,13 +451,13 @@ project dependencies, you can create a DStream from Kafka as
      {% highlight scala %} import org.apache.spark.streaming.kafka._ -KafkaUtils.createStream(streamingContext, kafkaParams, ...) +KafkaUtils.createStream(ssc, kafkaParams, ...) {% endhighlight %}
      {% highlight java %} -import org.apache.spark.streaming.kafka.* -KafkaUtils.createStream(javaStreamingContext, kafkaParams, ...); +import org.apache.spark.streaming.kafka.*; +KafkaUtils.createStream(jssc, kafkaParams, ...); {% endhighlight %}
      @@ -578,13 +586,14 @@ val runningCounts = pairs.updateStateByKey[Int](updateFunction _)
      {% highlight java %} +import com.google.common.base.Optional; Function2, Optional, Optional> updateFunction = new Function2, Optional, Optional>() { @Override public Optional call(List values, Optional state) { Integer newSum = ... // add the new values with the previous running count to get the new count - return Optional.of(newSum) + return Optional.of(newSum); } - } + }; {% endhighlight %} This is applied on a DStream containing words (say, the `pairs` DStream containing `(word, @@ -617,9 +626,9 @@ spam information (maybe generated with Spark as well) and then filtering based o
      {% highlight scala %} -val spamInfoRDD = sparkContext.hadoopFile(...) // RDD containing spam information +val spamInfoRDD = ssc.sparkContext.newAPIHadoopRDD(...) // RDD containing spam information -val cleanedDStream = inputDStream.transform(rdd => { +val cleanedDStream = wordCounts.transform(rdd => { rdd.join(spamInfoRDD).filter(...) // join data stream with spam information to do data cleaning ... }) @@ -629,13 +638,14 @@ val cleanedDStream = inputDStream.transform(rdd => {
      {% highlight java %} +import org.apache.spark.streaming.api.java.*; // RDD containing spam information -JavaPairRDD spamInfoRDD = javaSparkContext.hadoopFile(...); +final JavaPairRDD spamInfoRDD = jssc.sparkContext().newAPIHadoopRDD(...); -JavaPairDStream cleanedDStream = inputDStream.transform( +JavaPairDStream cleanedDStream = wordCounts.transform( new Function, JavaPairRDD>() { @Override public JavaPairRDD call(JavaPairRDD rdd) throws Exception { - rdd.join(spamInfoRDD).filter(...) // join data stream with spam information to do data cleaning + rdd.join(spamInfoRDD).filter(...); // join data stream with spam information to do data cleaning ... } }); @@ -684,7 +694,7 @@ operation `reduceByKeyAndWindow`. {% highlight scala %} // Reduce last 30 seconds of data, every 10 seconds -val windowedWordCounts = pairs.reduceByKeyAndWindow(_ + _, Seconds(30), Seconds(10)) +val windowedWordCounts = pairs.reduceByKeyAndWindow((a:Int,b:Int) => (a + b), Seconds(30), Seconds(10)) {% endhighlight %}
      @@ -699,7 +709,7 @@ Function2 reduceFunc = new Function2 windowedWordCounts = pair.reduceByKeyAndWindow(reduceFunc, new Duration(30000), new Duration(10000)); +JavaPairDStream windowedWordCounts = pairs.reduceByKeyAndWindow(reduceFunc, new Duration(30000), new Duration(10000)); {% endhighlight %}
      @@ -1087,7 +1097,7 @@ This behavior is made simple by using `JavaStreamingContext.getOrCreate`. This i {% highlight java %} // Create a factory object that can create a and setup a new JavaStreamingContext JavaStreamingContextFactory contextFactory = new JavaStreamingContextFactory() { - JavaStreamingContextFactory create() { + @Override public JavaStreamingContext create() { JavaStreamingContext jssc = new JavaStreamingContext(...); // new context JavaDStream lines = jssc.socketTextStream(...); // create DStreams ... From ce72c72aecdcf50e031f0dd1a5ef0e6dbf8f40b5 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 3 May 2014 13:23:52 -0700 Subject: [PATCH 230/641] [Bugfix] Tachyon file cleanup logical error Should lookup `shutdownDeleteTachyonPaths` instead of `shutdownDeletePaths`. Together with a minor style clean up: `find {...}.isDefined` to `exists {...}`. Author: Cheng Lian Closes #575 from liancheng/tachyonFix and squashes the following commits: deb8f31 [Cheng Lian] Fixed logical error in when cleanup Tachyon files and minor style cleanup --- core/src/main/scala/org/apache/spark/util/Utils.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 536a740140c03..acd7eef6d25ea 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -220,9 +220,9 @@ private[spark] object Utils extends Logging { def hasRootAsShutdownDeleteDir(file: File): Boolean = { val absolutePath = file.getAbsolutePath() val retval = shutdownDeletePaths.synchronized { - shutdownDeletePaths.find { path => + shutdownDeletePaths.exists { path => !absolutePath.equals(path) && absolutePath.startsWith(path) - }.isDefined + } } if (retval) { logInfo("path = " + file + ", already present as root for deletion.") @@ -235,10 +235,10 @@ private[spark] object Utils extends Logging { // paths - resulting in Exception and incomplete cleanup. def hasRootAsShutdownDeleteDir(file: TachyonFile): Boolean = { val absolutePath = file.getPath() - val retval = shutdownDeletePaths.synchronized { - shutdownDeletePaths.find { path => + val retval = shutdownDeleteTachyonPaths.synchronized { + shutdownDeleteTachyonPaths.exists { path => !absolutePath.equals(path) && absolutePath.startsWith(path) - }.isDefined + } } if (retval) { logInfo("path = " + file + ", already present as root for deletion.") From 34719ba32ed421701eaa08bd47ce953cd9267ad7 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sat, 3 May 2014 13:27:10 -0700 Subject: [PATCH 231/641] SPARK-1689 AppClient should indicate app is dead() when removed Previously, we indicated disconnected(), which keeps the application in a limbo state where it has no executors but thinks it will get them soon. This is a bug fix that hopefully can be included in 1.0. Author: Aaron Davidson Closes #605 from aarondav/appremoved and squashes the following commits: bea02a2 [Aaron Davidson] SPARK-1689 AppClient should indicate app is dead() when removed --- .../org/apache/spark/deploy/client/AppClient.scala | 12 +++++------- .../spark/deploy/client/AppClientListener.scala | 4 ++-- .../org/apache/spark/deploy/client/TestClient.scala | 4 ++-- .../cluster/SparkDeploySchedulerBackend.scala | 6 +++--- 4 files changed, 12 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 8901806de9262..57085fc337148 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -92,8 +92,7 @@ private[spark] class AppClient( if (registered) { retryTimer.cancel() } else if (retries >= REGISTRATION_RETRIES) { - logError("All masters are unresponsive! Giving up.") - markDead() + markDead("All masters are unresponsive! Giving up.") } else { tryRegisterAllMasters() } @@ -126,8 +125,7 @@ private[spark] class AppClient( listener.connected(appId) case ApplicationRemoved(message) => - logError("Master removed our application: %s; stopping client".format(message)) - markDisconnected() + markDead("Master removed our application: %s".format(message)) context.stop(self) case ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) => @@ -158,7 +156,7 @@ private[spark] class AppClient( logWarning(s"Could not connect to $address: $cause") case StopAppClient => - markDead() + markDead("Application has been stopped.") sender ! true context.stop(self) } @@ -173,9 +171,9 @@ private[spark] class AppClient( } } - def markDead() { + def markDead(reason: String) { if (!alreadyDead) { - listener.dead() + listener.dead(reason) alreadyDead = true } } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala index 1f20aa3dfa39b..e584952a9ad85 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala @@ -30,8 +30,8 @@ private[spark] trait AppClientListener { /** Disconnection may be a temporary state, as we fail over to a new Master. */ def disconnected(): Unit - /** Dead means that we couldn't find any Masters to connect to, and have given up. */ - def dead(): Unit + /** An application death is an unrecoverable failure condition. */ + def dead(reason: String): Unit def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 888dd45e93c6a..e15a87bd38fda 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -33,8 +33,8 @@ private[spark] object TestClient { System.exit(0) } - def dead() { - logInfo("Could not connect to master") + def dead(reason: String) { + logInfo("Application died with error: " + reason) System.exit(0) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 9544ca05dca70..cefa41729964a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -83,10 +83,10 @@ private[spark] class SparkDeploySchedulerBackend( } } - override def dead() { + override def dead(reason: String) { if (!stopping) { - logError("Spark cluster looks dead, giving up.") - scheduler.error("Spark cluster looks down") + logError("Application has been killed. Reason: " + reason) + scheduler.error(reason) } } From 4669a84ab10296e01a5fbbae1de9574b793b7ed5 Mon Sep 17 00:00:00 2001 From: "Allan Douglas R. de Oliveira" Date: Sat, 3 May 2014 16:52:19 -0700 Subject: [PATCH 232/641] EC2 configurable workers Added option to configure number of worker instances and to set SPARK_MASTER_OPTS Depends on: https://github.com/mesos/spark-ec2/pull/46 Author: Allan Douglas R. de Oliveira Closes #612 from douglaz/ec2_configurable_workers and squashes the following commits: d6c5d65 [Allan Douglas R. de Oliveira] Added master opts parameter 6c34671 [Allan Douglas R. de Oliveira] Use number of worker instances as string on template ba528b9 [Allan Douglas R. de Oliveira] Added SPARK_WORKER_INSTANCES parameter --- ec2/deploy.generic/root/spark-ec2/ec2-variables.sh | 2 ++ ec2/spark_ec2.py | 12 ++++++++++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh index 42e8faa26ed09..3570891be804e 100644 --- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh +++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh @@ -28,3 +28,5 @@ export SPARK_VERSION="{{spark_version}}" export SHARK_VERSION="{{shark_version}}" export HADOOP_MAJOR_VERSION="{{hadoop_major_version}}" export SWAP_MB="{{swap}}" +export SPARK_WORKER_INSTANCES="{{spark_worker_instances}}" +export SPARK_MASTER_OPTS="{{spark_master_opts}}" diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 31209a662bbe1..db393748a33bf 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -103,6 +103,12 @@ def parse_args(): help="When destroying a cluster, delete the security groups that were created") parser.add_option("--use-existing-master", action="store_true", default=False, help="Launch fresh slaves, but use an existing stopped master if possible") + parser.add_option("--worker-instances", type="int", default=1, + help="Number of instances per worker: variable SPARK_WORKER_INSTANCES (default: 1)") + parser.add_option("--master-opts", type="string", default="", + help="Extra options to give to master through SPARK_MASTER_OPTS variable (e.g -Dspark.worker.timeout=180)") + + (opts, args) = parser.parse_args() if len(args) != 2: @@ -223,7 +229,7 @@ def launch_cluster(conn, opts, cluster_name): sys.exit(1) if opts.key_pair is None: print >> stderr, "ERROR: Must provide a key pair name (-k) to use on instances." - sys.exit(1) + sys.exit(1) print "Setting up security groups..." master_group = get_or_make_group(conn, cluster_name + "-master") slave_group = get_or_make_group(conn, cluster_name + "-slaves") @@ -551,7 +557,9 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): "modules": '\n'.join(modules), "spark_version": spark_v, "shark_version": shark_v, - "hadoop_major_version": opts.hadoop_major_version + "hadoop_major_version": opts.hadoop_major_version, + "spark_worker_instances": "%d" % opts.worker_instances, + "spark_master_opts": opts.master_opts } # Create a temp directory in which we will place all the files to be From b295714708476b2904e205ac6dc58867e205546e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 3 May 2014 18:38:44 -0700 Subject: [PATCH 233/641] [SQL] Better logging when applying rules. Author: Michael Armbrust Closes #616 from marmbrus/ruleLogging and squashes the following commits: 39c09fe [Michael Armbrust] Fix off by one error. 5af3537 [Michael Armbrust] Better logging when applying rules. --- .../sql/catalyst/rules/RuleExecutor.scala | 28 ++++++++++++++++--- 1 file changed, 24 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index 9db96f89dd03c..e32adb76fe146 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -50,18 +50,18 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { var curPlan = plan batches.foreach { batch => + val batchStartPlan = curPlan var iteration = 1 var lastPlan = curPlan - curPlan = batch.rules.foldLeft(curPlan) { case (plan, rule) => rule(plan) } + var continue = true // Run until fix point (or the max number of iterations as specified in the strategy. - while (iteration < batch.strategy.maxIterations && !curPlan.fastEquals(lastPlan)) { - lastPlan = curPlan + while (continue) { curPlan = batch.rules.foldLeft(curPlan) { case (plan, rule) => val result = rule(plan) if (!result.fastEquals(plan)) { - logger.debug( + logger.trace( s""" |=== Applying Rule ${rule.ruleName} === |${sideBySide(plan.treeString, result.treeString).mkString("\n")} @@ -71,6 +71,26 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { result } iteration += 1 + if (iteration > batch.strategy.maxIterations) { + logger.info(s"Max iterations ($iteration) reached for batch ${batch.name}") + continue = false + } + + if (curPlan.fastEquals(lastPlan)) { + logger.trace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.") + continue = false + } + lastPlan = curPlan + } + + if (!batchStartPlan.fastEquals(curPlan)) { + logger.debug( + s""" + |=== Result of Batch ${batch.name} === + |${sideBySide(plan.treeString, curPlan.treeString).mkString("\n")} + """.stripMargin) + } else { + logger.trace(s"Batch ${batch.name} has no effect.") } } From 92b2902ca06eddd250bb0f50f365ed69f127a842 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 3 May 2014 23:13:51 -0700 Subject: [PATCH 234/641] Whitelist Hive Tests This is ready when Jenkins is. Author: Michael Armbrust Closes #596 from marmbrus/moreTests and squashes the following commits: 85be703 [Michael Armbrust] Blacklist MR required tests. 35bc311 [Michael Armbrust] Add hive golden answers. ede98fd [Michael Armbrust] More hive gitignore da096ea [Michael Armbrust] update whitelist --- .gitignore | 1 + ...on_table-0-8e765b54f15b948fc88392da69da283 | 0 ...n_table-1-aee4ce62fc2631423af0f569f4448353 | 0 ...n_table-2-b1feb4a197caf28d5223e72e10a91e78 | 0 ...ocation-0-1c3ace37d0bbb5c8033c48cde7680d21 | 1 + ...ncefile-0-7df8fcbfff1c967de72295b90c530776 | 0 ...ncefile-1-3114fdebb1e9b0830de9e9c3fff8a67e | 0 ...ncefile-2-80ec34a069bc561aa6dc87314391b131 | 4 ++ ...ncefile-3-699522b1ff3f38cf6a75b06d83820b87 | 0 ...ncefile-4-adc1ec67836b26b60d8547c4996bfd8f | 10 ++++ ...at_text-0-c84258297070d93009fabc59c1f40a1e | 0 ...at_text-1-6375f4ba7af19f94c6afc366dc75429e | 0 ...at_text-2-80ec34a069bc561aa6dc87314391b131 | 4 ++ ...at_text-3-699522b1ff3f38cf6a75b06d83820b87 | 0 ...at_text-4-adc1ec67836b26b60d8547c4996bfd8f | 10 ++++ ...input15-0-1570712216ce86428ee39974242ae961 | 0 ...input15-1-86edc12357bf278d5e601a654358c32f | 0 ...input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 | 2 + ...putddl1-0-52f161c97a2a02494d26ee0737b28364 | 0 ...putddl1-1-dfe05b7247284b326f39481b3d66a0c3 | 0 ...putddl1-2-b516db5218781d89aebd2e4813ea3660 | 0 ...putddl2-0-4519aea06db13ec6780f60b4d101d262 | 0 ...putddl2-1-f526507cae3cad09c69d20c7f0e36abe | 0 ...putddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 | 10 ++++ ...putddl3-0-a14253f6c752c9f5e9f56875152f8353 | 0 ...putddl3-1-f066827dd440213f306ab7d680bc8526 | 0 ...putddl3-2-73f945a673d2e388847c317f683f160c | 2 + ...ry_data-4-3ebc340f7f63740f8534706d42dd37ca | Bin 0 -> 114 bytes ...terjoin-1-8c68ce10095a8924c68c8ee4b3c0071a | 22 ++++++++ ...outerjoin-2-8e5d5472f2e214c091c879f6830a0c | 0 ...terjoin-3-381cc29f131d8caba744dd9fe6c121e7 | 22 ++++++++ ...terjoin-4-ce1ef910fff98f174931cc641f7cef3a | 1 + ...terjoin-5-82cbc3186de23f3a2411e9ab87c0008c | 0 ...terjoin-6-c6844de37be0926316f5fbf36a905306 | 22 ++++++++ ...orderby-0-3ca1c197c5209d2fc9423fe84ad66e7d | 0 ...orderby-1-be7423a5e77b7289153f74bc3fd97f44 | 0 ...rderby-10-767848077fc2929ccedfd28e50564b19 | 1 + ...rderby-11-39767502cbda75590c0c4b8fd089b793 | 0 ...rderby-12-7943485bcc72b2040c45c62e45ac8853 | 0 ...rderby-13-988591bf9ab008fdc4d71560aa57a736 | 32 ++++++++++++ ...rderby-14-d93951df2ffc18dc09ab0ba2d46f1823 | 48 ++++++++++++++++++ ...orderby-2-be7423a5e77b7289153f74bc3fd97f44 | 0 ...orderby-3-8ae9591fe39cd390619181e9664a92c1 | 1 + ...orderby-4-a928b93138e9c5547c40ff2024b2b4b6 | 1 + ...orderby-5-a58344acc57aaa38e2cb5f11a0576681 | 1 + ...orderby-6-670ec1b1f28d92d72a924c29d622aa8f | 0 ...orderby-7-7943485bcc72b2040c45c62e45ac8853 | 0 ...orderby-8-988591bf9ab008fdc4d71560aa57a736 | 32 ++++++++++++ ...orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 | 48 ++++++++++++++++++ ...apjoin9-0-31cd5ae86d250c1c873260c4b73d2336 | 0 ...apjoin9-1-c5c86975c400b3a1ec0f522b75635338 | 0 ...pjoin9-10-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...pjoin9-11-43d53504df013e6b35f81811138a167a | 1 + ...apjoin9-12-7cccbdffc32975f8935eeba14a28147 | 1 + ...pjoin9-13-d0eac0de35b5a7595e4567edec8e555d | 0 ...pjoin9-14-f0de4fd75ca13f379fbea16010dfd3f9 | 0 ...pjoin9-15-b89ea2173180c8ae423d856f943e061f | 1 + ...pjoin9-16-5645429104736d72293601247e874df7 | 0 ...pjoin9-17-1018a99eaede08dc2b427d3fb4b91d6b | 0 ...pjoin9-18-bda1fd3b5d0326dc3a41a8fb480b6c1c | 0 ...pjoin9-19-823e8f68baaa45d6b761b9b9890bb902 | 0 ...apjoin9-2-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...pjoin9-20-78ea4515eba2d8a79bb6895133a82051 | 0 ...apjoin9-3-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...apjoin9-4-b89ea2173180c8ae423d856f943e061f | 1 + ...apjoin9-5-94bc4acde9b9c9944e2eb9a9c57d3450 | 0 ...apjoin9-6-6c0963a1bdf2def39b79bea21a9cdda9 | 0 ...apjoin9-7-ba9bc547b5d2a4912f4f8567fea4009c | 0 ...apjoin9-8-d528ff47621b8c86b370f72512a7cd3c | 0 ...apjoin9-9-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ..._format-0-3b2fa9592648fc07c0d43e0d7d7f9411 | 0 ..._format-1-d498fb503b8f47db4741fdad3d266b4a | 0 ..._format-2-42119039bf8023f90b7f474f235c5dd5 | 1 + ..._format-3-77b57147024eb6b28cc9f525fdaab615 | 1 + ..._format-4-676cb274a770a6b9ca86df5dc7f912d4 | 0 ..._format-5-ef3052815ec41b5957627698ba06707b | 0 ..._format-6-891be0baec05e358a647dcca77724446 | 0 ..._format-7-208bcc9c918cbeb52907c8871be19cd5 | 0 ..._format-8-fb2e7127e07ad9f7e97ad3df3eba3a35 | 0 ..._format-9-433d5dbbcf75ff68d6897f31baa46841 | 1 + ...udf_hour-4-73bfac513b993dedbe143306865a44a | 1 + ..._substr-8-ba6ca6bac87ca99aabd60b8e76537ade | 1 + ..._substr-9-a9aa82080133620d017160f6a644455d | 1 + ...union25-0-f4e883ee4edf8fbb95efed8ae41cff1c | 0 ...union25-1-8f4a32f3c1551b4e4f22c3b776a92043 | 0 ...union25-2-f61103bb2045761ba95828898e63b92b | 0 .../execution/HiveCompatibilitySuite.scala | 43 ++++++++++++++-- 87 files changed, 328 insertions(+), 4 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/create_union_table-0-8e765b54f15b948fc88392da69da283 create mode 100644 sql/hive/src/test/resources/golden/create_union_table-1-aee4ce62fc2631423af0f569f4448353 create mode 100644 sql/hive/src/test/resources/golden/create_union_table-2-b1feb4a197caf28d5223e72e10a91e78 create mode 100644 sql/hive/src/test/resources/golden/ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21 create mode 100644 sql/hive/src/test/resources/golden/fileformat_sequencefile-0-7df8fcbfff1c967de72295b90c530776 create mode 100644 sql/hive/src/test/resources/golden/fileformat_sequencefile-1-3114fdebb1e9b0830de9e9c3fff8a67e create mode 100644 sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 create mode 100644 sql/hive/src/test/resources/golden/fileformat_sequencefile-3-699522b1ff3f38cf6a75b06d83820b87 create mode 100644 sql/hive/src/test/resources/golden/fileformat_sequencefile-4-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/fileformat_text-0-c84258297070d93009fabc59c1f40a1e create mode 100644 sql/hive/src/test/resources/golden/fileformat_text-1-6375f4ba7af19f94c6afc366dc75429e create mode 100644 sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 create mode 100644 sql/hive/src/test/resources/golden/fileformat_text-3-699522b1ff3f38cf6a75b06d83820b87 create mode 100644 sql/hive/src/test/resources/golden/fileformat_text-4-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/input15-0-1570712216ce86428ee39974242ae961 create mode 100644 sql/hive/src/test/resources/golden/input15-1-86edc12357bf278d5e601a654358c32f create mode 100644 sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 create mode 100644 sql/hive/src/test/resources/golden/inputddl1-0-52f161c97a2a02494d26ee0737b28364 create mode 100644 sql/hive/src/test/resources/golden/inputddl1-1-dfe05b7247284b326f39481b3d66a0c3 create mode 100644 sql/hive/src/test/resources/golden/inputddl1-2-b516db5218781d89aebd2e4813ea3660 create mode 100644 sql/hive/src/test/resources/golden/inputddl2-0-4519aea06db13ec6780f60b4d101d262 create mode 100644 sql/hive/src/test/resources/golden/inputddl2-1-f526507cae3cad09c69d20c7f0e36abe create mode 100644 sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 create mode 100644 sql/hive/src/test/resources/golden/inputddl3-0-a14253f6c752c9f5e9f56875152f8353 create mode 100644 sql/hive/src/test/resources/golden/inputddl3-1-f066827dd440213f306ab7d680bc8526 create mode 100644 sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c create mode 100644 sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca create mode 100644 sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-1-8c68ce10095a8924c68c8ee4b3c0071a create mode 100644 sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-2-8e5d5472f2e214c091c879f6830a0c create mode 100644 sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-3-381cc29f131d8caba744dd9fe6c121e7 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-4-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-5-82cbc3186de23f3a2411e9ab87c0008c create mode 100644 sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-6-c6844de37be0926316f5fbf36a905306 create mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-0-3ca1c197c5209d2fc9423fe84ad66e7d create mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-1-be7423a5e77b7289153f74bc3fd97f44 create mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-10-767848077fc2929ccedfd28e50564b19 create mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-11-39767502cbda75590c0c4b8fd089b793 create mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-12-7943485bcc72b2040c45c62e45ac8853 create mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736 create mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823 create mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-2-be7423a5e77b7289153f74bc3fd97f44 create mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-3-8ae9591fe39cd390619181e9664a92c1 create mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-4-a928b93138e9c5547c40ff2024b2b4b6 create mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-5-a58344acc57aaa38e2cb5f11a0576681 create mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-6-670ec1b1f28d92d72a924c29d622aa8f create mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-7-7943485bcc72b2040c45c62e45ac8853 create mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736 create mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-0-31cd5ae86d250c1c873260c4b73d2336 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-1-c5c86975c400b3a1ec0f522b75635338 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-10-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-11-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-12-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-13-d0eac0de35b5a7595e4567edec8e555d create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-14-f0de4fd75ca13f379fbea16010dfd3f9 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-15-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-16-5645429104736d72293601247e874df7 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-17-1018a99eaede08dc2b427d3fb4b91d6b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-18-bda1fd3b5d0326dc3a41a8fb480b6c1c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-19-823e8f68baaa45d6b761b9b9890bb902 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-2-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-20-78ea4515eba2d8a79bb6895133a82051 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-3-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-4-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-5-94bc4acde9b9c9944e2eb9a9c57d3450 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-6-6c0963a1bdf2def39b79bea21a9cdda9 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-7-ba9bc547b5d2a4912f4f8567fea4009c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-8-d528ff47621b8c86b370f72512a7cd3c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin9-9-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-0-3b2fa9592648fc07c0d43e0d7d7f9411 create mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-1-d498fb503b8f47db4741fdad3d266b4a create mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5 create mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615 create mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-4-676cb274a770a6b9ca86df5dc7f912d4 create mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-5-ef3052815ec41b5957627698ba06707b create mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-6-891be0baec05e358a647dcca77724446 create mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-7-208bcc9c918cbeb52907c8871be19cd5 create mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-8-fb2e7127e07ad9f7e97ad3df3eba3a35 create mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841 create mode 100644 sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a create mode 100644 sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade create mode 100644 sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d create mode 100644 sql/hive/src/test/resources/golden/union25-0-f4e883ee4edf8fbb95efed8ae41cff1c create mode 100644 sql/hive/src/test/resources/golden/union25-1-8f4a32f3c1551b4e4f22c3b776a92043 create mode 100644 sql/hive/src/test/resources/golden/union25-2-f61103bb2045761ba95828898e63b92b diff --git a/.gitignore b/.gitignore index a204456d42085..32b603f1bc84f 100644 --- a/.gitignore +++ b/.gitignore @@ -54,3 +54,4 @@ scalastyle.txt metastore_db/ metastore/ warehouse/ +TempStatsStore/ diff --git a/sql/hive/src/test/resources/golden/create_union_table-0-8e765b54f15b948fc88392da69da283 b/sql/hive/src/test/resources/golden/create_union_table-0-8e765b54f15b948fc88392da69da283 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_union_table-1-aee4ce62fc2631423af0f569f4448353 b/sql/hive/src/test/resources/golden/create_union_table-1-aee4ce62fc2631423af0f569f4448353 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_union_table-2-b1feb4a197caf28d5223e72e10a91e78 b/sql/hive/src/test/resources/golden/create_union_table-2-b1feb4a197caf28d5223e72e10a91e78 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21 b/sql/hive/src/test/resources/golden/ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-0-7df8fcbfff1c967de72295b90c530776 b/sql/hive/src/test/resources/golden/fileformat_sequencefile-0-7df8fcbfff1c967de72295b90c530776 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-1-3114fdebb1e9b0830de9e9c3fff8a67e b/sql/hive/src/test/resources/golden/fileformat_sequencefile-1-3114fdebb1e9b0830de9e9c3fff8a67e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 b/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 new file mode 100644 index 0000000000000..6280b32facd66 --- /dev/null +++ b/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 @@ -0,0 +1,4 @@ +key int None +value string None + +Detailed Table Information Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1398823397, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/dest1, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1398823397}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-3-699522b1ff3f38cf6a75b06d83820b87 b/sql/hive/src/test/resources/golden/fileformat_sequencefile-3-699522b1ff3f38cf6a75b06d83820b87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-4-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/fileformat_sequencefile-4-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..e614f37ecc8bf --- /dev/null +++ b/sql/hive/src/test/resources/golden/fileformat_sequencefile-4-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,10 @@ +0 val_0 +4 val_4 +8 val_8 +0 val_0 +0 val_0 +5 val_5 +5 val_5 +2 val_2 +5 val_5 +9 val_9 diff --git a/sql/hive/src/test/resources/golden/fileformat_text-0-c84258297070d93009fabc59c1f40a1e b/sql/hive/src/test/resources/golden/fileformat_text-0-c84258297070d93009fabc59c1f40a1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_text-1-6375f4ba7af19f94c6afc366dc75429e b/sql/hive/src/test/resources/golden/fileformat_text-1-6375f4ba7af19f94c6afc366dc75429e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 b/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 new file mode 100644 index 0000000000000..e793ec2f946e5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 @@ -0,0 +1,4 @@ +key int None +value string None + +Detailed Table Information Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1398823407, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/dest1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1398823407}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/fileformat_text-3-699522b1ff3f38cf6a75b06d83820b87 b/sql/hive/src/test/resources/golden/fileformat_text-3-699522b1ff3f38cf6a75b06d83820b87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_text-4-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/fileformat_text-4-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..e614f37ecc8bf --- /dev/null +++ b/sql/hive/src/test/resources/golden/fileformat_text-4-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,10 @@ +0 val_0 +4 val_4 +8 val_8 +0 val_0 +0 val_0 +5 val_5 +5 val_5 +2 val_2 +5 val_5 +9 val_9 diff --git a/sql/hive/src/test/resources/golden/input15-0-1570712216ce86428ee39974242ae961 b/sql/hive/src/test/resources/golden/input15-0-1570712216ce86428ee39974242ae961 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input15-1-86edc12357bf278d5e601a654358c32f b/sql/hive/src/test/resources/golden/input15-1-86edc12357bf278d5e601a654358c32f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 b/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 new file mode 100644 index 0000000000000..ded361eb294f0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 @@ -0,0 +1,2 @@ +key int None +value string None diff --git a/sql/hive/src/test/resources/golden/inputddl1-0-52f161c97a2a02494d26ee0737b28364 b/sql/hive/src/test/resources/golden/inputddl1-0-52f161c97a2a02494d26ee0737b28364 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl1-1-dfe05b7247284b326f39481b3d66a0c3 b/sql/hive/src/test/resources/golden/inputddl1-1-dfe05b7247284b326f39481b3d66a0c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl1-2-b516db5218781d89aebd2e4813ea3660 b/sql/hive/src/test/resources/golden/inputddl1-2-b516db5218781d89aebd2e4813ea3660 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl2-0-4519aea06db13ec6780f60b4d101d262 b/sql/hive/src/test/resources/golden/inputddl2-0-4519aea06db13ec6780f60b4d101d262 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl2-1-f526507cae3cad09c69d20c7f0e36abe b/sql/hive/src/test/resources/golden/inputddl2-1-f526507cae3cad09c69d20c7f0e36abe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 b/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 new file mode 100644 index 0000000000000..679d54cb5cb5e --- /dev/null +++ b/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 @@ -0,0 +1,10 @@ +key int None +value string None +ds string None +country string None + +# Partition Information +# col_name data_type comment + +ds string None +country string None diff --git a/sql/hive/src/test/resources/golden/inputddl3-0-a14253f6c752c9f5e9f56875152f8353 b/sql/hive/src/test/resources/golden/inputddl3-0-a14253f6c752c9f5e9f56875152f8353 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl3-1-f066827dd440213f306ab7d680bc8526 b/sql/hive/src/test/resources/golden/inputddl3-1-f066827dd440213f306ab7d680bc8526 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c b/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c new file mode 100644 index 0000000000000..ded361eb294f0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c @@ -0,0 +1,2 @@ +key int None +value string None diff --git a/sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca b/sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca new file mode 100644 index 0000000000000000000000000000000000000000..de5212a3c320f772b6a3a2b246202cd99d07c012 GIT binary patch literal 114 zcmXYpK@xx<31zZO;W_B|0{Q<~&`YzM%(UUWxDW~|hJHUCips`;RC1i NaN = true && 1.0 < NaN = false... which is wrong. // http://stackoverflow.com/a/1573715 @@ -176,6 +178,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { */ override def whiteList = Seq( "add_part_exist", + "add_part_multiple", "add_partition_no_whitelist", "add_partition_with_whitelist", "alias_casted_column", @@ -290,6 +293,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "disable_file_format_check", "drop_function", "drop_index", + "drop_multi_partitions", "drop_partitions_filter", "drop_partitions_filter2", "drop_partitions_filter3", @@ -302,6 +306,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "escape_orderby1", "escape_sortby1", "fetch_aggregation", + "fileformat_sequencefile", + "fileformat_text", "filter_join_breaktask", "filter_join_breaktask2", "groupby1", @@ -310,6 +316,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby1_map_nomap", "groupby1_map_skew", "groupby1_noskew", + "groupby2", + "groupby2_map", + "groupby2_map_skew", + "groupby2_noskew", "groupby4", "groupby4_map", "groupby4_map_skew", @@ -333,10 +343,12 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby8_noskew", "groupby9", "groupby_distinct_samekey", + "groupby_map_ppr", "groupby_multi_insert_common_distinct", "groupby_multi_single_reducer2", "groupby_mutli_insert_common_distinct", "groupby_neg_float", + "groupby_ppr", "groupby_sort_10", "groupby_sort_2", "groupby_sort_3", @@ -352,13 +364,17 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "inoutdriver", "input", "input0", + "input1", + "input10", "input11", "input11_limit", "input12", "input12_hadoop20", "input14", + "input15", "input19", "input1_limit", + "input2", "input21", "input22", "input23", @@ -367,6 +383,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "input26", "input28", "input2_limit", + "input3", + "input4", "input40", "input41", "input4_cb_delim", @@ -374,9 +392,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "input7", "input8", "input9", - "inputddl4", - "inputddl7", - "inputddl8", "input_limit", "input_part0", "input_part1", @@ -391,6 +406,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "input_part8", "input_part9", "input_testsequencefile", + "inputddl1", + "inputddl2", + "inputddl3", + "inputddl4", + "inputddl6", + "inputddl7", + "inputddl8", "insert1", "insert2_overwrite_partitions", "insert_compressed", @@ -452,7 +474,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "join_reorder4", "join_star", "join_view", + "lateral_view", "lateral_view_cp", + "lateral_view_outer", "lateral_view_ppd", "lineage1", "literal_double", @@ -463,6 +487,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "loadpart1", "louter_join_ppr", "mapjoin_distinct", + "mapjoin_filter_on_outerjoin", "mapjoin_mapjoin", "mapjoin_subquery", "mapjoin_subquery2", @@ -577,6 +602,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "skewjoinopt13", "skewjoinopt18", "skewjoinopt9", + "smb_mapjoin9", "smb_mapjoin_1", "smb_mapjoin_10", "smb_mapjoin_13", @@ -621,8 +647,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_10_trims", "udf2", "udf6", + "udf7", "udf8", "udf9", + "udf_E", + "udf_PI", "udf_abs", "udf_acos", "udf_add", @@ -646,6 +675,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_ceil", "udf_ceiling", "udf_concat", + "udf_concat_insert1", "udf_concat_insert2", "udf_concat_ws", "udf_conv", @@ -660,6 +690,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_div", "udf_double", "udf_E", + "udf_elt", "udf_exp", "udf_field", "udf_find_in_set", @@ -669,9 +700,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_from_unixtime", "udf_greaterthan", "udf_greaterthanorequal", + "udf_hash", "udf_hex", "udf_if", "udf_index", + "udf_instr", "udf_int", "udf_isnotnull", "udf_isnull", @@ -682,6 +715,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_lessthanorequal", "udf_like", "udf_ln", + "udf_locate", "udf_log", "udf_log10", "udf_log2", @@ -740,9 +774,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_trim", "udf_ucase", "udf_upper", - "udf_variance", "udf_var_pop", "udf_var_samp", + "udf_variance", "udf_weekofyear", "udf_when", "udf_xpath", @@ -768,6 +802,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "union22", "union23", "union24", + "union25", "union26", "union27", "union28", From fb0543224bcedb8ae3aab4a7ddcc6111a03378fe Mon Sep 17 00:00:00 2001 From: witgo Date: Sat, 3 May 2014 23:32:12 -0700 Subject: [PATCH 235/641] The default version of yarn is equal to the hadoop version This is a part of [PR 590](https://github.com/apache/spark/pull/590) Author: witgo Closes #626 from witgo/yarn_version and squashes the following commits: c390631 [witgo] restore the yarn dependency declarations f8a4ad8 [witgo] revert remove the dependency of avro in yarn-alpha 2df6cf5 [witgo] review commit a1d876a [witgo] review commit 20e7e3e [witgo] review commit c76763b [witgo] The default value of yarn.version is equal to hadoop.version --- bin/compute-classpath.sh | 1 + docs/building-with-maven.md | 9 ++++++--- pom.xml | 15 +++++++++------ yarn/pom.xml | 5 ++--- 4 files changed, 18 insertions(+), 12 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index b0218531e9eb8..70ac71e45db4b 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -44,6 +44,7 @@ if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SCALA_VERSION/classes" DEPS_ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar` CLASSPATH="$CLASSPATH:$DEPS_ASSEMBLY_JAR" diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index a5e5303467401..e447dfea3bac9 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -45,17 +45,20 @@ For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions wit For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you can enable the "yarn-alpha" or "yarn" profile and set the "hadoop.version", "yarn.version" property. Note that Hadoop 0.23.X requires a special `-Phadoop-0.23` profile: # Apache Hadoop 2.0.5-alpha - $ mvn -Pyarn-alpha -Dhadoop.version=2.0.5-alpha -Dyarn.version=2.0.5-alpha -DskipTests clean package + $ mvn -Pyarn-alpha -Dhadoop.version=2.0.5-alpha -DskipTests clean package # Cloudera CDH 4.2.0 with MapReduce v2 - $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-cdh4.2.0 -DskipTests clean package + $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -DskipTests clean package # Apache Hadoop 2.2.X (e.g. 2.2.0 as below) and newer - $ mvn -Pyarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -DskipTests clean package + $ mvn -Pyarn -Dhadoop.version=2.2.0 -DskipTests clean package # Apache Hadoop 0.23.x $ mvn -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.7 -Dyarn.version=0.23.7 -DskipTests clean package + # Different versions of HDFS and YARN. + $ mvn -Pyarn-alpha -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -DskipTests clean package + ## Spark Tests in Maven ## Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). Some of the require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. You can then run the tests with `mvn -Dhadoop.version=... test`. diff --git a/pom.xml b/pom.xml index 646753fe30301..ebd359a9de17f 100644 --- a/pom.xml +++ b/pom.xml @@ -16,7 +16,8 @@ ~ limitations under the License. --> - + 4.0.0 org.apache @@ -119,7 +120,7 @@ 1.2.17 1.0.4 2.4.1 - 0.23.7 + ${hadoop.version} 0.94.6 0.12.0 1.3.2 @@ -135,7 +136,8 @@ - maven-repo + maven-repo + Maven Repository http://repo.maven.apache.org/maven2 @@ -847,15 +849,16 @@ 0.23.7 - - yarn - org.apache.avro avro + + yarn + + diff --git a/yarn/pom.xml b/yarn/pom.xml index 02f36627431b9..e0094738266b1 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -28,7 +28,7 @@ yarn-parent_2.10 pom Spark Project YARN Parent POM - + org.apache.spark @@ -50,7 +50,6 @@ org.apache.hadoop hadoop-client - ${yarn.version} org.scalatest @@ -128,7 +127,7 @@ - + From e97a2e6717e75c70c5474f2d211682565909e557 Mon Sep 17 00:00:00 2001 From: Rahul Singhal Date: Sun, 4 May 2014 11:08:39 -0700 Subject: [PATCH 236/641] SPARK-1658: Correctly identify if maven is installed and working The current test is checking the exit code of "tail" rather than "mvn". This new check will make sure that mvn is installed and was able to execute the "version command". Author: Rahul Singhal Closes #580 from rahulsinghaliitd/SPARK-1658 and squashes the following commits: 83c0313 [Rahul Singhal] SPARK-1658: Correctly identify if maven is installed and working bf821b9 [Rahul Singhal] SPARK-1658: Correctly identify if maven is installed and working --- make-distribution.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index c05dcd89d90a7..dc8aa56a2e973 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -43,8 +43,9 @@ FWDIR="$(cd `dirname $0`; pwd)" DISTDIR="$FWDIR/dist" -VERSION=$(mvn help:evaluate -Dexpression=project.version | grep -v "INFO" | tail -n 1) -if [ $? == -1 ] ;then +set -o pipefail +VERSION=$(mvn help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1) +if [ $? != 0 ]; then echo -e "You need Maven installed to build Spark." echo -e "Download Maven from https://maven.apache.org." exit -1; From 0088cede592540f35f9aec0f24dc1d9bd690d878 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 4 May 2014 11:55:29 -0700 Subject: [PATCH 237/641] SPARK-1663. (Addendum) Fix signature of one version of JavaPairRDDStream.reduceByKeyAndWindow() It appears that one of these methods doesn't use `org.apache.spark.api.java.function.Function2` like all the others, but uses Scala's `Function2`. Author: Sean Owen Closes #633 from srowen/SPARK-1663.2 and squashes the following commits: 1e0232d [Sean Owen] Fix signature of one version of reduceByKeyAndWindow to use Java API Function2, as apparently intended --- .../org/apache/spark/streaming/api/java/JavaPairDStream.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index cb8e8f00a7b82..c4bdf01fa3744 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -262,7 +262,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval */ - def reduceByKeyAndWindow(reduceFunc: Function2[V, V, V], windowDuration: Duration) + def reduceByKeyAndWindow(reduceFunc: JFunction2[V, V, V], windowDuration: Duration) :JavaPairDStream[K, V] = { dstream.reduceByKeyAndWindow(reduceFunc, windowDuration) } From 0c98a8f6a761b941a9e0cf6fde6df8d0b8d24057 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 4 May 2014 12:22:23 -0700 Subject: [PATCH 238/641] SPARK-1703 Warn users if Spark is run on JRE6 but compiled with JDK7. This add some guards and good warning messages if users hit this issue. /cc @aarondav with whom I discussed parts of the design. Author: Patrick Wendell Closes #627 from pwendell/jdk6 and squashes the following commits: a38a958 [Patrick Wendell] Code review feedback 94e9f84 [Patrick Wendell] SPARK-1703 Warn users if Spark is run on JRE6 but compiled with JDK7. --- bin/compute-classpath.sh | 14 ++++++++++++++ bin/spark-class | 9 ++++++++- make-distribution.sh | 14 ++++++++++++++ 3 files changed, 36 insertions(+), 1 deletion(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 70ac71e45db4b..8dc547b379f50 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -32,6 +32,12 @@ CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH:$FWDIR/conf" ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION" +if [ -n "${JAVA_HOME}" ]; then + JAR_CMD="${JAVA_HOME}/bin/jar" +else + JAR_CMD="jar" +fi + # First check if we have a dependencies jar. If so, include binary classes with the deps jar if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes" @@ -55,6 +61,14 @@ else else ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*.jar` fi + jar_error_check=$($JAR_CMD -tf $ASSEMBLY_JAR org/apache/spark/SparkContext 2>&1) + if [[ "$jar_error_check" =~ "invalid CEN header" ]]; then + echo "Loading Spark jar with '$JAR_CMD' failed. " + echo "This is likely because Spark was compiled with Java 7 and run " + echo "with Java 6. (see SPARK-1703). Please use Java 7 to run Spark " + echo "or build Spark with Java 6." + exit 1 + fi CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" fi diff --git a/bin/spark-class b/bin/spark-class index e8160c8af64c1..72f8b9bf9a495 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -138,7 +138,14 @@ if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then fi # Compute classpath using external script -CLASSPATH=`$FWDIR/bin/compute-classpath.sh` +classpath_output=$($FWDIR/bin/compute-classpath.sh) +if [[ "$?" != "0" ]]; then + echo "$classpath_output" + exit 1 +else + CLASSPATH=$classpath_output +fi + if [[ "$1" =~ org.apache.spark.tools.* ]]; then CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" fi diff --git a/make-distribution.sh b/make-distribution.sh index dc8aa56a2e973..2ab64ff4598af 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -51,6 +51,20 @@ if [ $? != 0 ]; then exit -1; fi +if [ -z "${JAVA_HOME}" ]; then + echo "Error: JAVA_HOME is not set, cannot proceed." + exit -1 +fi + +JAVA_CMD=$JAVA_HOME/bin/java +JAVA_VERSION=$($JAVA_CMD -version 2>&1) +if ! [[ "$JAVA_VERSION" =~ "1.6" ]]; then + echo "Error: JAVA_HOME must point to a JDK 6 installation (see SPARK-1703)." + echo "Output from 'java -version' was:" + echo "$JAVA_VERSION" + exit -1 +fi + # Initialize defaults SPARK_HADOOP_VERSION=1.0.4 SPARK_YARN=false From f5041579ff573f988b673c2506fa4edc32f5ad84 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 4 May 2014 17:43:28 -0700 Subject: [PATCH 239/641] SPARK-1629. Addendum: Depend on commons lang3 (already used by tachyon) as it's used in ReplSuite, and return to use lang3 utility in Utils.scala For consideration. This was proposed in related discussion: https://github.com/apache/spark/pull/569 Author: Sean Owen Closes #635 from srowen/SPARK-1629.2 and squashes the following commits: a442b98 [Sean Owen] Depend on commons lang3 (already used by tachyon) as it's used in ReplSuite, and return to use lang3 utility in Utils.scala --- core/pom.xml | 4 ++++ core/src/main/scala/org/apache/spark/util/Utils.scala | 5 ++--- pom.xml | 5 +++++ project/SparkBuild.scala | 1 + 4 files changed, 12 insertions(+), 3 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 822b5b1dd7cc2..36c71e67b56aa 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -69,6 +69,10 @@ com.google.guava guava + + org.apache.commons + commons-lang3 + com.google.code.findbugs jsr305 diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index acd7eef6d25ea..bef4dab3d7cc1 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -32,6 +32,7 @@ import scala.util.Try import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.json4s._ import tachyon.client.{TachyonFile,TachyonFS} @@ -1073,9 +1074,7 @@ private[spark] object Utils extends Logging { /** * Return true if this is Windows. */ - def isWindows = { - Option(System.getProperty("os.name")).exists(_.startsWith("Windows")) - } + def isWindows = SystemUtils.IS_OS_WINDOWS /** * Indicates whether Spark is currently running unit tests. diff --git a/pom.xml b/pom.xml index ebd359a9de17f..7678043484a4c 100644 --- a/pom.xml +++ b/pom.xml @@ -221,6 +221,11 @@ guava 14.0.1 + + org.apache.commons + commons-lang3 + 3.3.2 + com.google.code.findbugs jsr305 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 51f7335111166..19aa3c0607a21 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -326,6 +326,7 @@ object SparkBuild extends Build { name := "spark-core", libraryDependencies ++= Seq( "com.google.guava" % "guava" % "14.0.1", + "org.apache.commons" % "commons-lang3" % "3.3.2", "com.google.code.findbugs" % "jsr305" % "1.3.9", "log4j" % "log4j" % "1.2.17", "org.slf4j" % "slf4j-api" % slf4jVersion, From d940e4c16aaa7b60daf1229a99bc4d3455c0240d Mon Sep 17 00:00:00 2001 From: witgo Date: Sun, 4 May 2014 17:48:52 -0700 Subject: [PATCH 240/641] SPARK-1693: Most of the tests throw a java.lang.SecurityException when s... ...park built for hadoop 2.3.0 , 2.4.0 Author: witgo Closes #628 from witgo/SPARK-1693_new and squashes the following commits: e3af968 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1693_new dc63905 [witgo] SPARK-1693: Most of the tests throw a java.lang.SecurityException when spark built for hadoop 2.3.0 , 2.4.0 --- pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pom.xml b/pom.xml index 7678043484a4c..08c3ac64436b9 100644 --- a/pom.xml +++ b/pom.xml @@ -601,6 +601,10 @@ org.jboss.netty netty + + javax.servlet + servlet-api + @@ -621,6 +625,10 @@ org.jboss.netty netty + + javax.servlet + servlet-api + From bcb9b7fd4a656f9a6741220a6623441567ded0a4 Mon Sep 17 00:00:00 2001 From: "Allan Douglas R. de Oliveira" Date: Sun, 4 May 2014 20:36:51 -0700 Subject: [PATCH 241/641] EC2 script should exit with non-zero code on UsageError This is specially import because some ssh errors are raised as UsageError, preventing an automated usage of the script from detecting the failure. Author: Allan Douglas R. de Oliveira Closes #638 from douglaz/ec2_exit_code_fix and squashes the following commits: 5915e6d [Allan Douglas R. de Oliveira] EC2 script should exit with non-zero code on UsageError --- ec2/spark_ec2.py | 1 + 1 file changed, 1 insertion(+) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index db393748a33bf..0f6d5a1fa81e5 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -814,6 +814,7 @@ def main(): real_main() except UsageError, e: print >> stderr, "\nError:\n", e + sys.exit(1) if __name__ == "__main__": From b48a55ae9ff2976c5fe6f5776a6d4659e828ee24 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Sun, 4 May 2014 20:51:53 -0700 Subject: [PATCH 242/641] SPARK-1710: spark-submit should print better errors than "InvocationTargetException" Catching the InvocationTargetException, printing getTargetException. Author: Sandeep Closes #630 from techaddict/SPARK-1710 and squashes the following commits: 834d79b [Sandeep] changes from srowen suggestions 109d604 [Sandeep] SPARK-1710: spark-submit should print better errors than "InvocationTargetException" --- .../scala/org/apache/spark/deploy/SparkSubmit.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index d131f1809c99e..fb30e8a70f682 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy import java.io.{File, PrintStream} +import java.lang.reflect.InvocationTargetException import java.net.{URI, URL} import scala.collection.mutable.{ArrayBuffer, HashMap, Map} @@ -137,7 +138,7 @@ object SparkSubmit { throw new Exception(msg) } } - + // Special flag to avoid deprecation warnings at the client sysProps("SPARK_SUBMIT") = "true" @@ -253,7 +254,14 @@ object SparkSubmit { val mainClass = Class.forName(childMainClass, true, loader) val mainMethod = mainClass.getMethod("main", new Array[String](0).getClass) - mainMethod.invoke(null, childArgs.toArray) + try { + mainMethod.invoke(null, childArgs.toArray) + } catch { + case e: InvocationTargetException => e.getCause match { + case cause: Throwable => throw cause + case null => throw e + } + } } private def addJarToClasspath(localJar: String, loader: ExecutorURLClassLoader) { From bb2bb0cf6efcb1d18846e6c4904f3cd2da452f9f Mon Sep 17 00:00:00 2001 From: msiddalingaiah Date: Sun, 4 May 2014 21:59:10 -0700 Subject: [PATCH 243/641] Address SPARK-1717 I tested the change locally with Spark 0.9.1, but I can't test with 1.0.0 because there was no AMI for it at the time. It's a trivial fix, so it shouldn't cause any problems. Author: msiddalingaiah Closes #641 from msiddalingaiah/master and squashes the following commits: a4f7404 [msiddalingaiah] Address SPARK-1717 --- ec2/spark_ec2.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 0f6d5a1fa81e5..0a8d6ca5d7519 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -612,7 +612,7 @@ def ssh_command(opts): return ['ssh'] + ssh_args(opts) -# Run a command on a host through ssh, retrying up to two times +# Run a command on a host through ssh, retrying up to five times # and then throwing an exception if ssh continues to fail. def ssh(host, opts, command): tries = 0 @@ -621,7 +621,7 @@ def ssh(host, opts, command): return subprocess.check_call( ssh_command(opts) + ['-t', '-t', '%s@%s' % (opts.user, host), stringify_command(command)]) except subprocess.CalledProcessError as e: - if (tries > 2): + if (tries > 5): # If this was an ssh failure, provide the user with hints. if e.returncode == 255: raise UsageError("Failed to SSH to remote host {0}.\nPlease check that you have provided the correct --identity-file and --key-pair parameters and try again.".format(host)) @@ -648,7 +648,7 @@ def ssh_write(host, opts, command, input): status = proc.wait() if status == 0: break - elif (tries > 2): + elif (tries > 5): raise RuntimeError("ssh_write failed with error %s" % proc.returncode) else: print >> stderr, "Error {0} while executing remote command, retrying after 30 seconds".format(status) From f2eb070acc81e60096ee8d4ddf8da2b24a11da72 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 5 May 2014 00:52:06 -0700 Subject: [PATCH 244/641] Updated doc for spark.closure.serializer to indicate only Java serializer work. See discussion from http://apache-spark-developers-list.1001551.n3.nabble.com/bug-using-kryo-as-closure-serializer-td6473.html Author: Reynold Xin Closes #642 from rxin/docs-ser and squashes the following commits: a507db5 [Reynold Xin] Use "Java" instead of default. 5eb8cdd [Reynold Xin] Updated doc for spark.closure.serializer to indicate only the default serializer work. --- docs/configuration.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index b078c7c1112c2..81ad895d90e61 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -289,8 +289,7 @@ Apart from these, the following properties are also available, and may be useful spark.closure.serializer org.apache.spark.serializer.
      JavaSerializer - Serializer class to use for closures. Generally Java is fine unless your distributed functions - (e.g. map functions) reference large objects in the driver program. + Serializer class to use for closures. Currently only the Java serializer is supported. From 73b0cbcc241cca3d318ff74340e80b02f884acbd Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 5 May 2014 10:33:49 -0700 Subject: [PATCH 245/641] SPARK-1556. jets3t dep doesn't update properly with newer Hadoop versions See related discussion at https://github.com/apache/spark/pull/468 This PR may still overstep what you have in mind, but let me put it on the table to start. Besides fixing the issue, it has one substantive change, and that is to manage Hadoop-specific things only in Hadoop-related profiles. This does _not_ remove `yarn.version`. - Moves the YARN and Hadoop profiles together in pom.xml. Sorry that this makes the diff a little hard to grok but the changes are only as follows. - Removes `hadoop.major.version` - Introduce `hadoop-2.2` and `hadoop-2.3` profiles to control Hadoop-specific changes: - like the protobuf version issue - this was only 'solved' now by enabling YARN for 2.2+, which is really an orthogonal issue - like the jets3t version issue now - Hadoop profiles set an appropriate default `hadoop.version`, that can be overridden - _(YARN profiles in the parent now only exist to add the sub-module)_ - Fixes the jets3t dependency issue - and makes it a runtime dependency - and centralizes config of this guy in the parent pom - Updates build docs - Updates SBT build too - and fixes a regex problem along the way Author: Sean Owen Closes #629 from srowen/SPARK-1556 and squashes the following commits: c3fa967 [Sean Owen] Fix hadoop-2.4 profile typo in doc a2105fd [Sean Owen] Add hadoop-2.4 profile and don't set hadoop.version in profiles 274f4f9 [Sean Owen] Make jets3t a runtime dependency, and bring its exclusion up into parent config bbed826 [Sean Owen] Use jets3t 0.9.0 for Hadoop 2.3+ (and correct similar regex issue in SBT build) f21f356 [Sean Owen] Build changes to set up for jets3t fix --- core/pom.xml | 6 --- docs/building-with-maven.md | 51 +++++++++++++++++----- pom.xml | 84 ++++++++++++++++++++++--------------- project/SparkBuild.scala | 5 ++- 4 files changed, 94 insertions(+), 52 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 36c71e67b56aa..c24c7be204087 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -38,12 +38,6 @@ net.java.dev.jets3t jets3t - - - commons-logging - commons-logging - - org.apache.curator diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index e447dfea3bac9..cac01ded60d94 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -29,9 +29,22 @@ You can fix this by setting the `MAVEN_OPTS` variable as discussed before. ## Specifying the Hadoop version ## -Because HDFS is not protocol-compatible across versions, if you want to read from HDFS, you'll need to build Spark against the specific HDFS version in your environment. You can do this through the "hadoop.version" property. If unset, Spark will build against Hadoop 1.0.4 by default. - -For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: +Because HDFS is not protocol-compatible across versions, if you want to read from HDFS, you'll need to build Spark against the specific HDFS version in your environment. You can do this through the "hadoop.version" property. If unset, Spark will build against Hadoop 1.0.4 by default. Note that certain build profiles are required for particular Hadoop versions: + + + + + + + + + + + + +
      Hadoop versionProfile required
      0.23.xhadoop-0.23
      1.x to 2.1.x(none)
      2.2.xhadoop-2.2
      2.3.xhadoop-2.3
      2.4.xhadoop-2.4
      + +For Apache Hadoop versions 1.x, Cloudera CDH "mr1" distributions, and other Hadoop versions without YARN, use: # Apache Hadoop 1.2.1 $ mvn -Dhadoop.version=1.2.1 -DskipTests clean package @@ -42,22 +55,40 @@ For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions wit # Apache Hadoop 0.23.x $ mvn -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package -For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you can enable the "yarn-alpha" or "yarn" profile and set the "hadoop.version", "yarn.version" property. Note that Hadoop 0.23.X requires a special `-Phadoop-0.23` profile: +For Apache Hadoop 2.x, 0.23.x, Cloudera CDH, and other Hadoop versions with YARN, you can enable the "yarn-alpha" or "yarn" profile and optionally set the "yarn.version" property if it is different from "hadoop.version". The additional build profile required depends on the YARN version: + + + + + + + + + +
      YARN versionProfile required
      0.23.x to 2.1.xyarn-alpha
      2.2.x and lateryarn
      + +Examples: # Apache Hadoop 2.0.5-alpha $ mvn -Pyarn-alpha -Dhadoop.version=2.0.5-alpha -DskipTests clean package - # Cloudera CDH 4.2.0 with MapReduce v2 + # Cloudera CDH 4.2.0 $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -DskipTests clean package - # Apache Hadoop 2.2.X (e.g. 2.2.0 as below) and newer - $ mvn -Pyarn -Dhadoop.version=2.2.0 -DskipTests clean package - # Apache Hadoop 0.23.x - $ mvn -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.7 -Dyarn.version=0.23.7 -DskipTests clean package + $ mvn -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package + + # Apache Hadoop 2.2.X + $ mvn -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -DskipTests clean package + + # Apache Hadoop 2.3.X + $ mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -DskipTests clean package + + # Apache Hadoop 2.4.X + $ mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package # Different versions of HDFS and YARN. - $ mvn -Pyarn-alpha -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -DskipTests clean package + $ mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -DskipTests clean package ## Spark Tests in Maven ## diff --git a/pom.xml b/pom.xml index 08c3ac64436b9..e4b5c36d69176 100644 --- a/pom.xml +++ b/pom.xml @@ -129,6 +129,7 @@ 0.3.6 3.0.0 1.7.4 + 0.7.1 64m 512m @@ -560,10 +561,18 @@
      + net.java.dev.jets3t jets3t - 0.7.1 + ${jets3t.version} + runtime + + + commons-logging + commons-logging + + org.apache.hadoop @@ -843,36 +852,6 @@ - - - hadoop-0.23 - - - org.apache.avro - avro - - - - - - yarn-alpha - - 2 - - 0.23.7 - - - - - org.apache.avro - avro - - - - yarn - - - @@ -907,17 +886,54 @@ + + - yarn + hadoop-0.23 + + + + org.apache.avro + avro + + + + + + hadoop-2.2 + + 2.5.0 + + + + + hadoop-2.3 - 2 - 2.2.0 2.5.0 + 0.9.0 + + + + hadoop-2.4 + + 2.5.0 + 0.9.0 + + + + + yarn-alpha yarn + + + yarn + + yarn + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 19aa3c0607a21..a2597e3e6ddd6 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -95,7 +95,7 @@ object SparkBuild extends Build { lazy val hadoopVersion = Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION) lazy val isNewHadoop = Properties.envOrNone("SPARK_IS_NEW_HADOOP") match { case None => { - val isNewHadoopVersion = "2.[2-9]+".r.findFirstIn(hadoopVersion).isDefined + val isNewHadoopVersion = "^2\\.[2-9]+".r.findFirstIn(hadoopVersion).isDefined (isNewHadoopVersion|| DEFAULT_IS_NEW_HADOOP) } case Some(v) => v.toBoolean @@ -297,6 +297,7 @@ object SparkBuild extends Build { val chillVersion = "0.3.6" val codahaleMetricsVersion = "3.0.0" val jblasVersion = "1.2.3" + val jets3tVersion = if ("^2\\.[3-9]+".r.findFirstIn(hadoopVersion).isDefined) "0.9.0" else "0.7.1" val jettyVersion = "8.1.14.v20131031" val hiveVersion = "0.12.0" val parquetVersion = "1.3.2" @@ -343,7 +344,7 @@ object SparkBuild extends Build { "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.13.0", "commons-net" % "commons-net" % "2.2", - "net.java.dev.jets3t" % "jets3t" % "0.7.1" excludeAll(excludeCommonsLogging), + "net.java.dev.jets3t" % "jets3t" % jets3tVersion excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm), "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeNetty), From 3292e2a71bfb5df5ba156cf7557747d164d12291 Mon Sep 17 00:00:00 2001 From: Bouke van der Bijl Date: Mon, 5 May 2014 11:19:35 -0700 Subject: [PATCH 246/641] SPARK-1721: Reset the thread classLoader in the Mesos Executor This is because Mesos calls it with a different environment or something, the result is that the Spark jar is missing and it can't load classes. This fixes http://apache-spark-user-list.1001560.n3.nabble.com/java-lang-ClassNotFoundException-spark-on-mesos-td3510.html I have no idea whether this is the right fix, I can only confirm that it fixes the issue for us. The `registered` method is called from mesos (https://github.com/apache/mesos/blob/765ff9bc2ac5a12d4362f8235b572a37d646390a/src/java/jni/org_apache_mesos_MesosExecutorDriver.cpp) I am unsure which commit caused this regression Author: Bouke van der Bijl Closes #620 from bouk/mesos-classloader-fix and squashes the following commits: c13eae0 [Bouke van der Bijl] Use getContextOrSparkClassLoader in SparkEnv and CompressionCodec --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 4 +--- .../src/main/scala/org/apache/spark/io/CompressionCodec.scala | 3 ++- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index bea435ec34ce9..d40ed27da5392 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -156,13 +156,11 @@ object SparkEnv extends Logging { conf.set("spark.driver.port", boundPort.toString) } - val classLoader = Thread.currentThread.getContextClassLoader - // Create an instance of the class named by the given Java system property, or by // 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) - val cls = Class.forName(name, true, classLoader) + val cls = Class.forName(name, true, Utils.getContextOrSparkClassLoader) // First try with the constructor that takes SparkConf. If we can't find one, // use a no-arg constructor instead. try { diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index e1a5ee316bb69..4b0fe1ab82999 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -24,6 +24,7 @@ import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.Utils /** * :: DeveloperApi :: @@ -49,7 +50,7 @@ private[spark] object CompressionCodec { } def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { - val ctor = Class.forName(codecName, true, Thread.currentThread.getContextClassLoader) + val ctor = Class.forName(codecName, true, Utils.getContextOrSparkClassLoader) .getConstructor(classOf[SparkConf]) ctor.newInstance(conf).asInstanceOf[CompressionCodec] } From a975a19f21e71f448b3fdb2ed4461e28ef439900 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 5 May 2014 15:28:19 -0700 Subject: [PATCH 247/641] [SPARK-1504], [SPARK-1505], [SPARK-1558] Updated Spark Streaming guide - SPARK-1558: Updated custom receiver guide to match it with the new API - SPARK-1504: Added deployment and monitoring subsection to streaming - SPARK-1505: Added migration guide for migrating from 0.9.x and below to Spark 1.0 - Updated various Java streaming examples to use JavaReceiverInputDStream to highlight the API change. - Removed the requirement for cleaner ttl from streaming guide Author: Tathagata Das Closes #652 from tdas/doc-fix and squashes the following commits: cb4f4b7 [Tathagata Das] Possible fix for flaky graceful shutdown test. ab71f7f [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into doc-fix 8d6ff9b [Tathagata Das] Addded migration guide to Spark Streaming. 7d171df [Tathagata Das] Added reference to JavaReceiverInputStream in examples and streaming guide. 49edd7c [Tathagata Das] Change java doc links to use Java docs. 11528d7 [Tathagata Das] Updated links on index page. ff80970 [Tathagata Das] More updates to streaming guide. 4dc42e9 [Tathagata Das] Added monitoring and other documentation in the streaming guide. 14c6564 [Tathagata Das] Updated custom receiver guide. --- docs/configuration.md | 6 +- docs/index.md | 6 +- docs/streaming-custom-receivers.md | 273 ++++++++++++------ docs/streaming-programming-guide.md | 200 ++++++++++--- .../examples/JavaCustomReceiver.java | 3 +- .../examples/JavaFlumeEventCount.java | 2 +- .../examples/JavaKafkaWordCount.java | 4 +- .../examples/JavaNetworkWordCount.java | 5 +- .../spark/streaming/receiver/Receiver.scala | 6 +- .../streaming/StreamingContextSuite.scala | 1 + 10 files changed, 360 insertions(+), 146 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 81ad895d90e61..d6f316ba5f773 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -462,7 +462,7 @@ Apart from these, the following properties are also available, and may be useful (infinite) Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks generated, etc.). - Periodic cleanups will ensure that metadata older than this duration will be forgetten. This is + Periodic cleanups will ensure that metadata older than this duration will be forgotten. This is useful for running Spark for many hours / days (for example, running 24/7 in case of Spark Streaming applications). Note that any RDD that persists in memory for more than this duration will be cleared as well. @@ -471,8 +471,8 @@ Apart from these, the following properties are also available, and may be useful spark.streaming.blockInterval 200 - Duration (milliseconds) of how long to batch new objects coming from network receivers used - in Spark Streaming. + Interval (milliseconds) at which data received by Spark Streaming receivers is coalesced + into blocks of data before storing them in Spark. diff --git a/docs/index.md b/docs/index.md index 6fc9a4f03bb12..2daa208b3b903 100644 --- a/docs/index.md +++ b/docs/index.md @@ -112,10 +112,10 @@ Note that on Windows, you need to set the environment variables on separate line * [Shark](http://shark.cs.berkeley.edu): Apache Hive over Spark * [Mailing Lists](http://spark.apache.org/mailing-lists.html): ask questions about Spark here * [AMP Camps](http://ampcamp.berkeley.edu/): a series of training camps at UC Berkeley that featured talks and - exercises about Spark, Shark, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/agenda-2012), - [slides](http://ampcamp.berkeley.edu/agenda-2012) and [exercises](http://ampcamp.berkeley.edu/exercises-2012) are + exercises about Spark, Shark, Spark Streaming, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/3/), + [slides](http://ampcamp.berkeley.edu/3/) and [exercises](http://ampcamp.berkeley.edu/3/exercises/) are available online for free. -* [Code Examples](http://spark.apache.org/examples.html): more are also available in the [examples subfolder](https://github.com/apache/spark/tree/master/examples/src/main/scala/) of Spark +* [Code Examples](http://spark.apache.org/examples.html): more are also available in the [examples subfolder](https://github.com/apache/spark/tree/master/examples/src/main/scala/org/apache/spark/) of Spark * [Paper Describing Spark](http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf) * [Paper Describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index 3cfa4516cc12d..a2dc3a8961dfc 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -3,126 +3,219 @@ layout: global title: Spark Streaming Custom Receivers --- -A "Spark Streaming" receiver can be a simple network stream, streams of messages from a message queue, files etc. A receiver can also assume roles more than just receiving data like filtering, preprocessing, to name a few of the possibilities. The api to plug-in any user defined custom receiver is thus provided to encourage development of receivers which may be well suited to ones specific need. +Spark Streaming can receive streaming data from any arbitrary data source beyond +the one's for which it has in-built support (that is, beyond Flume, Kafka, files, sockets, etc.). +This requires the developer to implement a *receiver* that is customized for receiving data from +the concerned data source. This guide walks through the process of implementing a custom receiver +and using it in a Spark Streaming application. + +### Implementing a Custom Receiver + +This starts with implementing a [Receiver](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver). +A custom receiver must extend this abstract class by implementing two methods +- `onStart()`: Things to do to start receiving data. +- `onStop()`: Things to do to stop receiving data. + +Note that `onStart()` and `onStop()` must not block indefinitely. Typically, onStart() would start the threads +that responsible for receiving the data and `onStop()` would ensure that the receiving by those threads +are stopped. The receiving threads can also use `isStopped()`, a `Receiver` method, to check whether they +should stop receiving data. + +Once the data is received, that data can be stored inside Spark +by calling `store(data)`, which is a method provided by the +[Receiver](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver) class. +There are number of flavours of `store()` which allow you store the received data +record-at-a-time or as whole collection of objects / serialized bytes. + +Any exception in the receiving threads should be caught and handled properly to avoid silent +failures of the receiver. `restart()` will restart the receiver by +asynchronously calling `onStop()` and then calling `onStart()` after a delay. +`stop()` will call `onStop()` and terminate the receiver. Also, `reportError()` +reports a error message to the driver (visible in the logs and UI) without stopping / restarting +the receiver. + +The following is a custom receiver that receives a stream of text over a socket. It treats +'\n' delimited lines in the text stream as records and stores them with Spark. If the receiving thread +has any error connecting or receiving, the receiver is restarted to make another attempt to connect. + +
      +
      -This guide shows the programming model and features by walking through a simple sample receiver and corresponding Spark Streaming application. +{% highlight scala %} -### Writing a Simple Receiver +class CustomReceiver(host: String, port: Int) + extends Receiver[String](StorageLevel.MEMORY_AND_DISK_2) with Logging { + + def onStart() { + // Start the thread that receives data over a connection + new Thread("Socket Receiver") { + override def run() { receive() } + }.start() + } + + def onStop() { + // There is nothing much to do as the thread calling receive() + // is designed to stop by itself isStopped() returns false + } + + /** Create a socket connection and receive data until receiver is stopped */ + private def receive() { + var socket: Socket = null + var userInput: String = null + try { + // Connect to host:port + socket = new Socket(host, port) + + // Until stopped or connection broken continue reading + val reader = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8")) + userInput = reader.readLine() + while(!isStopped && userInput != null) { + store(userInput) + userInput = reader.readLine() + } + reader.close() + socket.close() + + // Restart in an attempt to connect again when server is active again + restart("Trying to connect again") + } catch { + case e: java.net.ConnectException => + // restart if could not connect to server + restart("Error connecting to " + host + ":" + port, e) + case t: Throwable => + // restart if there is any other error + restart("Error receiving data", t) + } + } +} -This starts with implementing [NetworkReceiver](api/scala/index.html#org.apache.spark.streaming.dstream.NetworkReceiver). +{% endhighlight %} -The following is a simple socket text-stream receiver. +
      +
      + +{% highlight java %} + +public class JavaCustomReceiver extends Receiver { + + String host = null; + int port = -1; + + public JavaCustomReceiver(String host_ , int port_) { + super(StorageLevel.MEMORY_AND_DISK_2()); + host = host_; + port = port_; + } + + public void onStart() { + // Start the thread that receives data over a connection + new Thread() { + @Override public void run() { + receive(); + } + }.start(); + } + + public void onStop() { + // There is nothing much to do as the thread calling receive() + // is designed to stop by itself isStopped() returns false + } + + /** Create a socket connection and receive data until receiver is stopped */ + private void receive() { + Socket socket = null; + String userInput = null; + + try { + // connect to the server + socket = new Socket(host, port); + + BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream())); + + // Until stopped or connection broken continue reading + while (!isStopped() && (userInput = reader.readLine()) != null) { + System.out.println("Received data '" + userInput + "'"); + store(userInput); + } + reader.close(); + socket.close(); + + // Restart in an attempt to connect again when server is active again + restart("Trying to connect again"); + } catch(ConnectException ce) { + // restart if could not connect to server + restart("Could not connect", ce); + } catch(Throwable t) { + // restart if there is any other error + restart("Error receiving data", t); + } + } +} -{% highlight scala %} - class SocketTextStreamReceiver(host: String, port: Int) - extends NetworkReceiver[String] - { - protected lazy val blocksGenerator: BlockGenerator = - new BlockGenerator(StorageLevel.MEMORY_ONLY_SER_2) - - protected def onStart() = { - blocksGenerator.start() - val socket = new Socket(host, port) - val dataInputStream = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8")) - var data: String = dataInputStream.readLine() - while (data != null) { - blocksGenerator += data - data = dataInputStream.readLine() - } - } - - protected def onStop() { - blocksGenerator.stop() - } - } {% endhighlight %} +
      +
      -All we did here is extended NetworkReceiver and called blockGenerator's API method (i.e. +=) to push our blocks of data. Please refer to scala-docs of NetworkReceiver for more details. +### Using the custom receiver in a Spark Streaming application -### An Actor as Receiver +The custom receiver can be used in a Spark Streaming application by using +`streamingContext.receiverStream()`. This will create +input DStream using data received by the instance of custom receiver, as shown below -This starts with implementing [Actor](#References) - -Following is a simple socket text-stream receiver, which is appearently overly simplified using Akka's socket.io api. +
      +
      {% highlight scala %} - class SocketTextStreamReceiver (host:String, - port:Int, - bytesToString: ByteString => String) extends Actor with Receiver { - - override def preStart = IOManager(context.system).connect(host, port) - - def receive = { - case IO.Read(socket, bytes) => pushBlock(bytesToString(bytes)) - } - - } +// Assuming ssc is the StreamingContext +val customReceiverStream = ssc.receiverStream(new CustomReceiver(host, port)) +val words = lines.flatMap(_.split(" ")) +... {% endhighlight %} -All we did here is mixed in trait Receiver and called pushBlock api method to push our blocks of data. Please refer to scala-docs of Receiver for more details. - -### A Sample Spark Application +The full source code is in the example [CustomReceiver.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala). -* First create a Spark streaming context with master url and batchduration. +
      +
      -{% highlight scala %} - val ssc = new StreamingContext(master, "WordCountCustomStreamSource", - Seconds(batchDuration)) +{% highlight java %} +// Assuming ssc is the JavaStreamingContext +JavaDStream customReceiverStream = ssc.receiverStream(new JavaCustomReceiver(host, port)); +JavaDStream words = lines.flatMap(new FlatMapFunction() { ... }); +... {% endhighlight %} -* Plug-in the custom receiver into the spark streaming context and create a DStream. +The full source code is in the example [JavaCustomReceiver.java](https://github.com/apache/spark/blob/master/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java). -{% highlight scala %} - val lines = ssc.networkStream[String](new SocketTextStreamReceiver( - "localhost", 8445)) -{% endhighlight %} +
      +
      -* OR Plug-in the actor as receiver into the spark streaming context and create a DStream. -{% highlight scala %} - val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver( - "localhost",8445, z => z.utf8String)),"SocketReceiver") -{% endhighlight %} -* Process it. +### Implementing and Using a Custom Actor-based Receiver -{% highlight scala %} - val words = lines.flatMap(_.split(" ")) - val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) +Custom [Akka Actors](http://doc.akka.io/docs/akka/2.2.4/scala/actors.html) can also be used to +receive data. The [`ActorHelper`](api/scala/index.html#org.apache.spark.streaming.receiver.ActorHelper) +trait can be applied on any Akka actor, which allows received data to be stored in Spark using + `store(...)` methods. The supervisor strategy of this actor can be configured to handle failures, etc. - wordCounts.print() - ssc.start() +{% highlight scala %} +class CustomActor extends Actor with ActorHelper { + def receive = { + case data: String => store(data) + } +} {% endhighlight %} -* After processing it, stream can be tested using the netcat utility. - - $ nc -l localhost 8445 - hello world - hello hello - - -## Multiple Homogeneous/Heterogeneous Receivers. - -A DStream union operation is provided for taking union on multiple input streams. +And a new input stream can be created with this custom actor as {% highlight scala %} - val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver( - "localhost",8445, z => z.utf8String)),"SocketReceiver") - - // Another socket stream receiver - val lines2 = ssc.actorStream[String](Props(new SocketTextStreamReceiver( - "localhost",8446, z => z.utf8String)),"SocketReceiver") - - val union = lines.union(lines2) +// Assuming ssc is the StreamingContext +val lines = ssc.actorStream[String](Props(new CustomActor()), "CustomReceiver") {% endhighlight %} -Above stream can be easily process as described earlier. - -_A more comprehensive example is provided in the spark streaming examples_ +See [ActorWordCount.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala) +for an end-to-end example. -## References -1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html) -2.[NetworkReceiver](api/scala/index.html#org.apache.spark.streaming.dstream.NetworkReceiver) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index b22bb45828b20..e8b718b303560 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -136,7 +136,7 @@ The complete code can be found in the Spark Streaming example
      First, we create a -[JavaStreamingContext](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) object, +[JavaStreamingContext](api/java/org/apache/spark/streaming/api/java/JavaStreamingContext.html) object, which is the main entry point for all streaming functionality. Besides Spark's configuration, we specify that any DStream would be processed in 1 second batches. @@ -155,7 +155,7 @@ by specifying the IP address and port of the data server. {% highlight java %} // Create a DStream that will connect to serverIP:serverPort, like localhost:9999 -JavaDStream lines = jssc.socketTextStream("localhost", 9999); +JavaReceiverInputDStream lines = jssc.socketTextStream("localhost", 9999); {% endhighlight %} This `lines` DStream represents the stream of data that will be received from the data @@ -863,6 +863,51 @@ For DStreams that must be checkpointed (that is, DStreams created by `updateStat `reduceByKeyAndWindow` with inverse function), the checkpoint interval of the DStream is by default set to a multiple of the DStream's sliding interval such that its at least 10 seconds. +## Deployment +A Spark Streaming application is deployed on a cluster in the same way as any other Spark application. +Please refer to the [deployment guide](cluster-overview.html) for more details. + +If a running Spark Streaming application needs to be upgraded (with new application code), then +there are two possible mechanism. + +- The upgraded Spark Streaming application is started and run in parallel to the existing application. +Once the new one (receiving the same data as the old one) has been warmed up and ready +for prime time, the old one be can be brought down. Note that this can be done for data sources that support +sending the data to two destinations (i.e., the earlier and upgraded applications). + +- The existing application is shutdown gracefully (see +[`StreamingContext.stop(...)`](api/scala/index.html#org.apache.spark.streaming.StreamingContext) +or [`JavaStreamingContext.stop(...)`](api/java/org/apache/spark/streaming/api/java/JavaStreamingContext.html) +for graceful shutdown options) which ensure data that have been received is completely +processed before shutdown. Then the +upgraded application can be started, which will start processing from the same point where the earlier +application left off. Note that this can be done only with input sources that support source-side buffering +(like Kafka, and Flume) as data needs to be buffered while the previous application down and +the upgraded application is not yet up. + +## Monitoring +Beyond Spark's [monitoring capabilities](monitoring.html), there are additional capabilities +specific to Spark Streaming. When a StreamingContext is used, the +[Spark web UI](monitoring.html#web-interfaces) shows +an additional `Streaming` tab which shows statistics about running receivers (whether +receivers are active, number of records received, receiver error, etc.) +and completed batches (batch processing times, queueing delays, etc.). This can be used to +monitor the progress of the streaming application. + +The following two metrics in web UI is particularly important - +*Processing Time* and *Scheduling Delay* (under *Batch Processing Statistics*). The first is the +time to process each batch of data, and the second is the time a batch waits in a queue +for the processing of previous batches to finish. If the batch processing time is consistently more +than the batch interval and/or the queueing delay keeps increasing, then it indicates the system is +not able to process the batches as fast they are being generated and falling behind. +In that case, consider +[reducing](#reducing-the-processing-time-of-each-batch) the batch processing time. + +The progress of a Spark Streaming program can also be monitored using the +[StreamingListener](api/scala/index.html#org.apache.spark.scheduler.StreamingListener) interface, +which allows you to get receiver status and processing times. Note that this is a developer API +and it is likely to be improved upon (i.e., more information reported) in the future. + *************************************************************************************************** # Performance Tuning @@ -875,7 +920,8 @@ improve the performance of you application. At a high level, you need to conside Reducing the processing time of each batch of data by efficiently using cluster resources.
    • - Setting the right batch size such that the data processing can keep up with the data ingestion. + Setting the right batch size such that the batches of data can be processed as fast as they + are received (that is, data processing keeps up with the data ingestion).
    • @@ -884,7 +930,30 @@ There are a number of optimizations that can be done in Spark to minimize the pr each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section highlights some of the most important ones. -### Level of Parallelism +### Level of Parallelism in Data Receiving +Receiving data over the network (like Kafka, Flume, socket, etc.) requires the data to deserialized +and stored in Spark. If the data receiving becomes a bottleneck in the system, then consider +parallelizing the data receiving. Note that each input DStream +creates a single receiver (running on a worker machine) that receives a single stream of data. +Receiving multiple data streams can therefore be achieved by creating multiple input DStreams +and configuring them to receive different partitions of the data stream from the source(s). +For example, a single Kafka input stream receiving two topics of data can be split into two +Kafka input streams, each receiving only one topic. This would run two receivers on two workers, +thus allowing data to received in parallel, and increasing overall throughput. + +Another parameter that should be considered is the receiver's blocking interval. For most receivers, +the received data is coalesced together into large blocks of data before storing inside Spark's memory. +The number of blocks in each batch determines the number of tasks that will be used to process those +the received data in a map-like transformation. This blocking interval is determined by the +[configuration parameter](configuration.html) `spark.streaming.blockInterval` and the default value +is 200 milliseconds. + +An alternative to receiving data with multiple input streams / receivers is to explicitly repartition +the input data stream (using `inputStream.repartition()`). +This distributes the received batches of data across all the machines in the cluster +before further processing. + +### Level of Parallelism in Data Processing Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of @@ -921,16 +990,22 @@ These changes may reduce batch processing time by 100s of milliseconds, thus allowing sub-second batch size to be viable. ## Setting the Right Batch Size -For a Spark Streaming application running on a cluster to be stable, the processing of the data -streams must keep up with the rate of ingestion of the data streams. Depending on the type of -computation, the batch size used may have significant impact on the rate of ingestion that can be -sustained by the Spark Streaming application on a fixed cluster resources. For example, let us +For a Spark Streaming application running on a cluster to be stable, the system should be able to +process data as fast as it is being received. In other words, batches of data should be processed +as fast as they are being generated. Whether this is true for an application can be found by +[monitoring](#monitoring) the processing times in the streaming web UI, where the batch +processing time should be less than the batch interval. + +Depending on the nature of the streaming +computation, the batch interval used may have significant impact on the data rates that can be +sustained by the application on a fixed set of cluster resources. For example, let us consider the earlier WordCountNetwork example. For a particular data rate, the system may be able -to keep up with reporting word counts every 2 seconds (i.e., batch size of 2 seconds), but not -every 500 milliseconds. +to keep up with reporting word counts every 2 seconds (i.e., batch interval of 2 seconds), but not +every 500 milliseconds. So the batch interval needs to be set such that the expected data rate in +production can be sustained. A good approach to figure out the right batch size for your application is to test it with a -conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system +conservative batch interval (say, 5-10 seconds) and a low data rate. To verify whether the system is able to keep up with data rate, you can check the value of the end-to-end delay experienced by each processed batch (either look for "Total delay" in Spark driver log4j logs, or use the [StreamingListener](api/scala/index.html#org.apache.spark.streaming.scheduler.StreamingListener) @@ -942,29 +1017,6 @@ data rate and/or reducing the batch size. Note that momentary increase in the de temporary data rate increases maybe fine as long as the delay reduces back to a low value (i.e., less than batch size). -## 24/7 Operation -By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). -But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic -cleanup of it metadata. This can be enabled by setting the -[configuration property](configuration.html#spark-properties) `spark.cleaner.ttl` to the number of -seconds you want any metadata to persist. For example, setting `spark.cleaner.ttl` to 600 would -cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. -Note, that this property needs to be set before the SparkContext is created. - -This value is closely tied with any window operation that is being used. Any window operation -would require the input data to be persisted in memory for at least the duration of the window. -Hence it is necessary to set the delay to at least the value of the largest window operation used -in the Spark Streaming application. If this delay is set too low, the application will throw an -exception saying so. - -## Monitoring -Besides Spark's in-built [monitoring capabilities](monitoring.html), -the progress of a Spark Streaming program can also be monitored using the [StreamingListener] -(api/scala/index.html#org.apache.spark.scheduler.StreamingListener) interface, -which allows you to get statistics of batch processing times, queueing delays, -and total end-to-end delays. Note that this is still an experimental API and it is likely to be -improved upon (i.e., more information reported) in the future. - ## Memory Tuning Tuning the memory usage and GC behavior of Spark applications have been discussed in great detail in the [Tuning Guide](tuning.html). It is recommended that you read that. In this section, @@ -1249,18 +1301,80 @@ in the file. This is what the sequence of outputs would be with and without a dr If the driver had crashed in the middle of the processing of time 3, then it will process time 3 and output 30 after recovery. +*************************************************************************************************** + +# Migration Guide from 0.9.1 or below to 1.x +Between Spark 0.9.1 and Spark 1.0, there were a few API changes made to ensure future API stability. +This section elaborates the steps required to migrate your existing code to 1.0. + +**Input DStreams**: All operations that create an input stream (e.g., `StreamingContext.socketStream`, +`FlumeUtils.createStream`, etc.) now returns +[InputDStream](api/scala/index.html#org.apache.spark.streaming.dstream.InputDStream) / +[ReceiverInputDStream](api/scala/index.html#org.apache.spark.streaming.dstream.ReceiverInputDStream) +(instead of DStream) for Scala, and [JavaInputDStream](api/java/org/apache/spark/streaming/api/java/JavaInputDStream.html) / +[JavaPairInputDStream](api/java/org/apache/spark/streaming/api/java/JavaPairInputDStream.html) / +[JavaReceiverInputDStream](api/java/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.html) / +[JavaPairReceiverInputDStream](api/java/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.html) +(instead of JavaDStream) for Java. This ensures that functionality specific to input streams can +be added to these classes in the future without breaking binary compatibility. +Note that your existing Spark Streaming applications should not require any change +(as these new classes are subclasses of DStream/JavaDStream) but may require recompilation with Spark 1.0. + +**Custom Network Receivers**: Since the release to Spark Streaming, custom network receivers could be defined +in Scala using the class NetworkReceiver. However, the API was limited in terms of error handling +and reporting, and could not be used from Java. Starting Spark 1.0, this class has been +replaced by [Receiver](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver) which has +the following advantages. + +* Methods like `stop` and `restart` have been added to for better control of the lifecycle of a receiver. See +the [custom receiver guide](streaming-custom-receiver.html) for more details. +* Custom receivers can be implemented using both Scala and Java. + +To migrate your existing custom receivers from the earlier NetworkReceiver to the new Receiver, you have +to do the following. + +* Make your custom receiver class extend +[`org.apache.spark.streaming.receiver.Receiver`](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver) +instead of `org.apache.spark.streaming.dstream.NetworkReceiver`. +* Earlier, a BlockGenerator object had to be created by the custom receiver, to which received data was +added for being stored in Spark. It had to be explicitly started and stopped from `onStart()` and `onStop()` +methods. The new Receiver class makes this unnecessary as it adds a set of methods named `store()` +that can be called to store the data in Spark. So, to migrate your custom network receiver, remove any +BlockGenerator object (does not exist any more in Spark 1.0 anyway), and use `store(...)` methods on +received data. + +**Actor-based Receivers**: Data could have been received using any Akka Actors by extending the actor class with +`org.apache.spark.streaming.receivers.Receiver` trait. This has been renamed to +[`org.apache.spark.streaming.receiver.ActorHelper`](api/scala/index.html#org.apache.spark.streaming.receiver.ActorHelper) +and the `pushBlock(...)` methods to store received data has been renamed to `store(...)`. Other helper classes in +the `org.apache.spark.streaming.receivers` package were also moved +to [`org.apache.spark.streaming.receiver`](api/scala/index.html#org.apache.spark.streaming.receiver.package) +package and renamed for better clarity. + +*************************************************************************************************** + # Where to Go from Here * API documentation - - Main docs of StreamingContext and DStreams in [Scala](api/scala/index.html#org.apache.spark.streaming.package) - and [Java](api/scala/index.html#org.apache.spark.streaming.api.java.package) - - Additional docs for - [Kafka](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$), - [Flume](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$), - [Twitter](api/scala/index.html#org.apache.spark.streaming.twitter.TwitterUtils$), - [ZeroMQ](api/scala/index.html#org.apache.spark.streaming.zeromq.ZeroMQUtils$), and - [MQTT](api/scala/index.html#org.apache.spark.streaming.mqtt.MQTTUtils$) + - Scala docs + * [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) and + [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) + * [KafkaUtils](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$), + [FlumeUtils](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$), + [TwitterUtils](api/scala/index.html#org.apache.spark.streaming.twitter.TwitterUtils$), + [ZeroMQUtils](api/scala/index.html#org.apache.spark.streaming.zeromq.ZeroMQUtils$), and + [MQTTUtils](api/scala/index.html#org.apache.spark.streaming.mqtt.MQTTUtils$) + - Java docs + * [JavaStreamingContext](api/java/org/apache/spark/streaming/api/java/JavaStreamingContext.html), + [JavaDStream](api/java/org/apache/spark/streaming/api/java/JavaDStream.html) and + [PairJavaDStream](api/java/org/apache/spark/streaming/api/java/PairJavaDStream.html) + * [KafkaUtils](api/java/org/apache/spark/streaming/kafka/KafkaUtils.html), + [FlumeUtils](api/java/org/apache/spark/streaming/flume/FlumeUtils.html), + [TwitterUtils](api/java/org/apache/spark/streaming/twitter/TwitterUtils.html), + [ZeroMQUtils](api/java/org/apache/spark/streaming/zeromq/ZeroMQUtils.html), and + [MQTTUtils](api/java/org/apache/spark/streaming/mqtt/MQTTUtils.html) * More examples in [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples) and [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/streaming/examples) -* [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) describing Spark Streaming. +* [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) and +[video](http://youtu.be/g171ndOHgJ0) describing Spark Streaming. diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java index a94fa621dc328..e36c7800be23e 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java @@ -26,6 +26,7 @@ import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.apache.spark.streaming.receiver.Receiver; import scala.Tuple2; @@ -69,7 +70,7 @@ public static void main(String[] args) { // Create a input stream with the custom receiver on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') - JavaDStream lines = ssc.receiverStream( + JavaReceiverInputDStream lines = ssc.receiverStream( new JavaCustomReceiver(args[1], Integer.parseInt(args[2]))); JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java index f061001dd264d..c59f7538f8a82 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java @@ -58,7 +58,7 @@ public static void main(String[] args) { JavaStreamingContext ssc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval, System.getenv("SPARK_HOME"), JavaStreamingContext.jarOfClass(JavaFlumeEventCount.class)); - JavaDStream flumeStream = FlumeUtils.createStream(ssc, "localhost", port); + JavaReceiverInputDStream flumeStream = FlumeUtils.createStream(ssc, "localhost", port); flumeStream.count(); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index d704be08d6945..8da9bcd05ac38 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -29,6 +29,7 @@ import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaPairReceiverInputDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.apache.spark.streaming.kafka.KafkaUtils; import scala.Tuple2; @@ -73,7 +74,8 @@ public static void main(String[] args) { topicMap.put(topic, numThreads); } - JavaPairDStream messages = KafkaUtils.createStream(jssc, args[1], args[2], topicMap); + JavaPairReceiverInputDStream messages = + KafkaUtils.createStream(jssc, args[1], args[2], topicMap); JavaDStream lines = messages.map(new Function, String>() { @Override diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java index 0cc9d0ae1a08e..098c329ff6808 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java @@ -18,6 +18,7 @@ package org.apache.spark.streaming.examples; import com.google.common.collect.Lists; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import scala.Tuple2; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function2; @@ -57,9 +58,9 @@ public static void main(String[] args) { new Duration(1000), System.getenv("SPARK_HOME"), JavaStreamingContext.jarOfClass(JavaNetworkWordCount.class)); - // Create a NetworkInputDStream on target ip:port and count the + // Create a JavaReceiverInputDStream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') - JavaDStream lines = ssc.socketTextStream(args[1], Integer.parseInt(args[2])); + JavaReceiverInputDStream lines = ssc.socketTextStream(args[1], Integer.parseInt(args[2])); JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala index b310c22b3ab78..5acf8a9a811ee 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -28,9 +28,11 @@ import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: * Abstract class of a receiver that can be run on worker nodes to receive external data. A - * custom receiver can be defined by defining the functions onStart() and onStop(). onStart() + * custom receiver can be defined by defining the functions `onStart()` and `onStop()`. `onStart()` * should define the setup steps necessary to start receiving data, - * and onStop() should define the cleanup steps necessary to stop receiving data. + * and `onStop()` should define the cleanup steps necessary to stop receiving data. + * Exceptions while receiving can be handled either by restarting the receiver with `restart(...)` + * or stopped completely by `stop(...)` or * * A custom receiver in Scala would look like this. * diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index ee0bc8b7d6a71..cd86019f63e7e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -183,6 +183,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w "Received records = " + TestReceiver.counter.get() + ", " + "processed records = " + runningCount ) + Thread.sleep(100) } } From cf0a8f0204bb8acdaf441b03c924c278fef08e28 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 5 May 2014 16:28:07 -0700 Subject: [PATCH 248/641] [SPARK-1681] Include datanucleus jars in Spark Hive distribution This copies the datanucleus jars over from `lib_managed` into `dist/lib`, if any. The `CLASSPATH` must also be updated to reflect this change. Author: Andrew Or Closes #610 from andrewor14/hive-distribution and squashes the following commits: a4bc96f [Andrew Or] Rename search path in jar error check fa205e1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into hive-distribution 7855f58 [Andrew Or] Have jar command respect JAVA_HOME + check for jar errors both cases c16bbfd [Andrew Or] Merge branch 'master' of github.com:apache/spark into hive-distribution 32f6826 [Andrew Or] Leave the double colons 940a1bb [Andrew Or] Add back 2>/dev/null 58357cc [Andrew Or] Include datanucleus jars in Spark distribution built with Hive support --- bin/compute-classpath.sh | 58 +++++++++++++++++++++++----------------- make-distribution.sh | 11 +++++--- 2 files changed, 40 insertions(+), 29 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 8dc547b379f50..7df43a555d562 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -32,8 +32,8 @@ CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH:$FWDIR/conf" ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION" -if [ -n "${JAVA_HOME}" ]; then - JAR_CMD="${JAVA_HOME}/bin/jar" +if [ -n "$JAVA_HOME" ]; then + JAR_CMD="$JAVA_HOME/bin/jar" else JAR_CMD="jar" fi @@ -52,40 +52,48 @@ if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SCALA_VERSION/classes" - DEPS_ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar` - CLASSPATH="$CLASSPATH:$DEPS_ASSEMBLY_JAR" + ASSEMBLY_JAR=$(ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar 2>/dev/null) else # Else use spark-assembly jar from either RELEASE or assembly directory if [ -f "$FWDIR/RELEASE" ]; then - ASSEMBLY_JAR=`ls "$FWDIR"/lib/spark-assembly*hadoop*.jar` + ASSEMBLY_JAR=$(ls "$FWDIR"/lib/spark-assembly*hadoop*.jar 2>/dev/null) else - ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*.jar` + ASSEMBLY_JAR=$(ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*.jar 2>/dev/null) fi - jar_error_check=$($JAR_CMD -tf $ASSEMBLY_JAR org/apache/spark/SparkContext 2>&1) - if [[ "$jar_error_check" =~ "invalid CEN header" ]]; then - echo "Loading Spark jar with '$JAR_CMD' failed. " - echo "This is likely because Spark was compiled with Java 7 and run " - echo "with Java 6. (see SPARK-1703). Please use Java 7 to run Spark " - echo "or build Spark with Java 6." - exit 1 - fi - CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" fi +# Verify that versions of java used to build the jars and run Spark are compatible +jar_error_check=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" nonexistent/class/path 2>&1) +if [[ "$jar_error_check" =~ "invalid CEN header" ]]; then + echo "Loading Spark jar with '$JAR_CMD' failed. " + echo "This is likely because Spark was compiled with Java 7 and run " + echo "with Java 6. (see SPARK-1703). Please use Java 7 to run Spark " + echo "or build Spark with Java 6." + exit 1 +fi + +CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" + # When Hive support is needed, Datanucleus jars must be included on the classpath. -# Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. +# Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. # Both sbt and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is # built with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark # assembly is built for Hive, before actually populating the CLASSPATH with the jars. # Note that this check order is faster (by up to half a second) in the case where Hive is not used. -num_datanucleus_jars=$(ls "$FWDIR"/lib_managed/jars/ 2>/dev/null | grep "datanucleus-.*\\.jar" | wc -l) -if [ $num_datanucleus_jars -gt 0 ]; then - AN_ASSEMBLY_JAR=${ASSEMBLY_JAR:-$DEPS_ASSEMBLY_JAR} - num_hive_files=$(jar tvf "$AN_ASSEMBLY_JAR" org/apache/hadoop/hive/ql/exec 2>/dev/null | wc -l) - if [ $num_hive_files -gt 0 ]; then +if [ -f "$FWDIR/RELEASE" ]; then + datanucleus_dir="$FWDIR"/lib +else + datanucleus_dir="$FWDIR"/lib_managed/jars +fi + +datanucleus_jars=$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar") +datanucleus_jars=$(echo "$datanucleus_jars" | tr "\n" : | sed s/:$//g) + +if [ -n "$datanucleus_jars" ]; then + hive_files=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" org/apache/hadoop/hive/ql/exec 2>/dev/null) + if [ -n "$hive_files" ]; then echo "Spark assembly has been built with Hive, including Datanucleus jars on classpath" 1>&2 - DATANUCLEUSJARS=$(echo "$FWDIR/lib_managed/jars"/datanucleus-*.jar | tr " " :) - CLASSPATH=$CLASSPATH:$DATANUCLEUSJARS + CLASSPATH="$CLASSPATH:$datanucleus_jars" fi fi @@ -105,10 +113,10 @@ fi # Add hadoop conf dir if given -- otherwise FileSystem.*, etc fail ! # Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts # the configurtion files. -if [ "x" != "x$HADOOP_CONF_DIR" ]; then +if [ -n "$HADOOP_CONF_DIR" ]; then CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR" fi -if [ "x" != "x$YARN_CONF_DIR" ]; then +if [ -n "$YARN_CONF_DIR" ]; then CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" fi diff --git a/make-distribution.sh b/make-distribution.sh index 2ab64ff4598af..ff18d01e7a616 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -51,13 +51,13 @@ if [ $? != 0 ]; then exit -1; fi -if [ -z "${JAVA_HOME}" ]; then +if [ -z "$JAVA_HOME" ]; then echo "Error: JAVA_HOME is not set, cannot proceed." exit -1 fi -JAVA_CMD=$JAVA_HOME/bin/java -JAVA_VERSION=$($JAVA_CMD -version 2>&1) +JAVA_CMD="$JAVA_HOME"/bin/java +JAVA_VERSION=$("$JAVA_CMD" -version 2>&1) if ! [[ "$JAVA_VERSION" =~ "1.6" ]]; then echo "Error: JAVA_HOME must point to a JDK 6 installation (see SPARK-1703)." echo "Output from 'java -version' was:" @@ -162,6 +162,10 @@ echo "Spark $VERSION built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" cp $FWDIR/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" +if [ "$SPARK_HIVE" == "true" ]; then + cp $FWDIR/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/" +fi + # Copy other things mkdir "$DISTDIR"/conf cp "$FWDIR"/conf/*.template "$DISTDIR"/conf @@ -170,7 +174,6 @@ cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/python" "$DISTDIR" cp -r "$FWDIR/sbin" "$DISTDIR" - # Download and copy in tachyon, if requested if [ "$SPARK_TACHYON" == "true" ]; then TACHYON_VERSION="0.4.1" From 8e724dcbad411c533e4f0cd053aa878e8a60694d Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 5 May 2014 18:26:34 -0700 Subject: [PATCH 249/641] SPARK-1728. JavaRDDLike.mapPartitionsWithIndex requires ClassTag Author: Sandy Ryza Closes #657 from sryza/sandy-spark-1728 and squashes the following commits: 4751443 [Sandy Ryza] SPARK-1728. JavaRDDLike.mapPartitionsWithIndex requires ClassTag --- .../main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 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 af06d1dca94f4..619bfd75be8eb 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 @@ -75,11 +75,11 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return a new RDD by applying a function to each partition of this RDD, while tracking the index * of the original partition. */ - def mapPartitionsWithIndex[R: ClassTag]( + def mapPartitionsWithIndex[R]( f: JFunction2[java.lang.Integer, java.util.Iterator[T], java.util.Iterator[R]], preservesPartitioning: Boolean = false): JavaRDD[R] = new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))), - preservesPartitioning)) + preservesPartitioning)(fakeClassTag))(fakeClassTag) /** * Return a new RDD by applying a function to all elements of this RDD. From ea10b3126167af3f50f7c2a70e1d942e839fcb66 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 5 May 2014 18:32:14 -0700 Subject: [PATCH 250/641] Expose SparkListeners and relevant classes as DeveloperApi Hopefully this can go into 1.0, as a few people on the user list have asked for this. Author: Andrew Or Closes #648 from andrewor14/expose-listeners and squashes the following commits: e45e1ef [Andrew Or] Add missing colons (minor) 350d643 [Andrew Or] Expose SparkListeners and relevant classes as DeveloperApi --- .../org/apache/spark/storage/BlockId.scala | 24 +++++++++++++------ .../apache/spark/storage/BlockManagerId.scala | 15 +++++++----- .../storage/BlockManagerMasterActor.scala | 4 +++- .../apache/spark/storage/StorageLevel.scala | 17 +++++++------ .../spark/storage/StorageStatusListener.scala | 7 ++++-- .../apache/spark/storage/StorageUtils.scala | 9 +++++-- .../apache/spark/ui/env/EnvironmentTab.scala | 5 +++- .../apache/spark/ui/exec/ExecutorsTab.scala | 5 +++- .../spark/ui/jobs/ExecutorSummary.scala | 10 ++++++-- .../spark/ui/jobs/JobProgressListener.scala | 8 +++++-- .../apache/spark/ui/storage/StorageTab.scala | 7 ++++-- 11 files changed, 78 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index cffea28fbf794..42ec181b00bb3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -19,14 +19,18 @@ package org.apache.spark.storage import java.util.UUID +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * Identifies a particular Block of data, usually associated with a single file. * A Block can be uniquely identified by its filename, but each type of Block has a different * set of keys which produce its unique name. * * If your BlockId should be serializable, be sure to add it to the BlockId.apply() method. */ -private[spark] sealed abstract class BlockId { +@DeveloperApi +sealed abstract class BlockId { /** A globally unique identifier for this Block. Can be used for ser/de. */ def name: String @@ -44,24 +48,29 @@ private[spark] sealed abstract class BlockId { } } -private[spark] case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { +@DeveloperApi +case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { def name = "rdd_" + rddId + "_" + splitIndex } -private[spark] case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) +@DeveloperApi +case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId } -private[spark] case class BroadcastBlockId(broadcastId: Long, field: String = "") extends BlockId { +@DeveloperApi +case class BroadcastBlockId(broadcastId: Long, field: String = "") extends BlockId { def name = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field) } -private[spark] case class TaskResultBlockId(taskId: Long) extends BlockId { +@DeveloperApi +case class TaskResultBlockId(taskId: Long) extends BlockId { def name = "taskresult_" + taskId } -private[spark] case class StreamBlockId(streamId: Int, uniqueId: Long) extends BlockId { +@DeveloperApi +case class StreamBlockId(streamId: Int, uniqueId: Long) extends BlockId { def name = "input-" + streamId + "-" + uniqueId } @@ -75,7 +84,8 @@ private[spark] case class TestBlockId(id: String) extends BlockId { def name = "test_" + id } -private[spark] object BlockId { +@DeveloperApi +object BlockId { val RDD = "rdd_([0-9]+)_([0-9]+)".r val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index be537d77309bc..b1585bd8199d1 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -20,17 +20,20 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import java.util.concurrent.ConcurrentHashMap +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils /** + * :: DeveloperApi :: * This class represent an unique identifier for a BlockManager. - * The first 2 constructors of this class is made private to ensure that - * BlockManagerId objects can be created only using the apply method in - * the companion object. This allows de-duplication of ID objects. - * Also, constructor parameters are private to ensure that parameters cannot - * be modified from outside this class. + * + * The first 2 constructors of this class is made private to ensure that BlockManagerId objects + * can be created only using the apply method in the companion object. This allows de-duplication + * of ID objects. Also, constructor parameters are private to ensure that parameters cannot be + * modified from outside this class. */ -private[spark] class BlockManagerId private ( +@DeveloperApi +class BlockManagerId private ( private var executorId_ : String, private var host_ : String, private var port_ : Int, diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 63fa5d3eb6541..98fa0df6ec289 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -28,6 +28,7 @@ import akka.actor.{Actor, ActorRef, Cancellable} import akka.pattern.ask import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -411,7 +412,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus } } -private[spark] case class BlockStatus( +@DeveloperApi +case class BlockStatus( storageLevel: StorageLevel, memSize: Long, diskSize: Long, diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index c9a52e0366d93..363de93e067b8 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -22,14 +22,17 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import org.apache.spark.annotation.DeveloperApi /** + * :: DeveloperApi :: * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, * or Tachyon, whether to drop the RDD to disk if it falls out of memory or Tachyon , whether to * keep the data in memory in a serialized format, and whether to replicate the RDD partitions on * multiple nodes. + * * The [[org.apache.spark.storage.StorageLevel$]] singleton object contains some static constants * for commonly useful storage levels. To create your own storage level object, use the * factory method of the singleton object (`StorageLevel(...)`). */ +@DeveloperApi class StorageLevel private( private var useDisk_ : Boolean, private var useMemory_ : Boolean, @@ -54,9 +57,9 @@ class StorageLevel private( assert(replication < 40, "Replication restricted to be less than 40 for calculating hashcodes") if (useOffHeap) { - require(useDisk == false, "Off-heap storage level does not support using disk") - require(useMemory == false, "Off-heap storage level does not support using heap memory") - require(deserialized == false, "Off-heap storage level does not support deserialized storage") + require(!useDisk, "Off-heap storage level does not support using disk") + require(!useMemory, "Off-heap storage level does not support using heap memory") + require(!deserialized, "Off-heap storage level does not support deserialized storage") require(replication == 1, "Off-heap storage level does not support multiple replication") } @@ -146,7 +149,7 @@ object StorageLevel { /** * :: DeveloperApi :: - * Create a new StorageLevel object without setting useOffHeap + * Create a new StorageLevel object without setting useOffHeap. */ @DeveloperApi def apply(useDisk: Boolean, useMemory: Boolean, useOffHeap: Boolean, @@ -155,7 +158,7 @@ object StorageLevel { /** * :: DeveloperApi :: - * Create a new StorageLevel object + * Create a new StorageLevel object. */ @DeveloperApi def apply(useDisk: Boolean, useMemory: Boolean, @@ -164,7 +167,7 @@ object StorageLevel { /** * :: DeveloperApi :: - * Create a new StorageLevel object from its integer representation + * Create a new StorageLevel object from its integer representation. */ @DeveloperApi def apply(flags: Int, replication: Int): StorageLevel = @@ -172,7 +175,7 @@ object StorageLevel { /** * :: DeveloperApi :: - * Read StorageLevel object from ObjectInput stream + * Read StorageLevel object from ObjectInput stream. */ @DeveloperApi def apply(in: ObjectInput): StorageLevel = { diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index 7a174959037be..a6e6627d54e01 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -19,12 +19,15 @@ package org.apache.spark.storage import scala.collection.mutable +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ /** - * A SparkListener that maintains executor storage status + * :: DeveloperApi :: + * A SparkListener that maintains executor storage status. */ -private[spark] class StorageStatusListener extends SparkListener { +@DeveloperApi +class StorageStatusListener extends SparkListener { private val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() def storageStatusList = executorIdToStorageStatus.values.toSeq diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 1eddd1cdc483b..6f3252a2f6d31 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -21,9 +21,14 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.SparkContext +import org.apache.spark.annotation.DeveloperApi -/** Storage information for each BlockManager. */ -private[spark] class StorageStatus( +/** + * :: DeveloperApi :: + * Storage information for each BlockManager. + */ +@DeveloperApi +class StorageStatus( val blockManagerId: BlockManagerId, val maxMem: Long, val blocks: mutable.Map[BlockId, BlockStatus] = mutable.Map.empty) { diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala index 03b46e1bd59af..bbbe55ecf44a1 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala @@ -17,6 +17,7 @@ package org.apache.spark.ui.env +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.ui._ @@ -30,9 +31,11 @@ private[ui] class EnvironmentTab(parent: SparkUI) extends WebUITab(parent, "envi } /** + * :: DeveloperApi :: * A SparkListener that prepares information to be displayed on the EnvironmentTab */ -private[ui] class EnvironmentListener extends SparkListener { +@DeveloperApi +class EnvironmentListener extends SparkListener { var jvmInformation = Seq[(String, String)]() var sparkProperties = Seq[(String, String)]() var systemProperties = Seq[(String, String)]() diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 5678bf34ac730..91d37b835b19d 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -20,6 +20,7 @@ package org.apache.spark.ui.exec import scala.collection.mutable.HashMap import org.apache.spark.ExceptionFailure +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.{SparkUI, WebUITab} @@ -34,9 +35,11 @@ private[ui] class ExecutorsTab(parent: SparkUI) extends WebUITab(parent, "execut } /** + * :: DeveloperApi :: * A SparkListener that prepares information to be displayed on the ExecutorsTab */ -private[ui] class ExecutorsListener(storageStatusListener: StorageStatusListener) +@DeveloperApi +class ExecutorsListener(storageStatusListener: StorageStatusListener) extends SparkListener { val executorToTasksActive = HashMap[String, Int]() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala index 1dfe1d4f1fa11..2aaf6329b792d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala @@ -17,8 +17,14 @@ package org.apache.spark.ui.jobs -/** class for reporting aggregated metrics for each executors in stageUI */ -private[ui] class ExecutorSummary { +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Class for reporting aggregated metrics for each executor in stage UI. + */ +@DeveloperApi +class ExecutorSummary { var taskTime : Long = 0 var failedTasks : Int = 0 var succeededTasks : Int = 0 diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 0db4afa701b41..396cbcbc8d268 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -20,19 +20,22 @@ package org.apache.spark.ui.jobs import scala.collection.mutable.{HashMap, ListBuffer} import org.apache.spark.{ExceptionFailure, SparkConf, SparkContext, Success} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId /** + * :: DeveloperApi :: * Tracks task-level information to be displayed in the UI. * * All access to the data structures in this class must be synchronized on the * class, since the UI thread and the EventBus loop may otherwise be reading and * updating the internal data structures concurrently. */ -private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { +@DeveloperApi +class JobProgressListener(conf: SparkConf) extends SparkListener { import JobProgressListener._ @@ -246,7 +249,8 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { } -private[ui] case class TaskUIData( +@DeveloperApi +case class TaskUIData( taskInfo: TaskInfo, taskMetrics: Option[TaskMetrics] = None, exception: Option[ExceptionFailure] = None) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 07ec297841f20..c4bb7aab50393 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -19,6 +19,7 @@ package org.apache.spark.ui.storage import scala.collection.mutable +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ui._ import org.apache.spark.scheduler._ import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} @@ -35,9 +36,11 @@ private[ui] class StorageTab(parent: SparkUI) extends WebUITab(parent, "storage" } /** - * A SparkListener that prepares information to be displayed on the BlockManagerUI + * :: DeveloperApi :: + * A SparkListener that prepares information to be displayed on the BlockManagerUI. */ -private[ui] class StorageListener(storageStatusListener: StorageStatusListener) +@DeveloperApi +class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener { private val _rddInfoMap = mutable.Map[Int, RDDInfo]() From 98750a74daf7e2b873da85d2d5067f47e3bbdc4e Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 5 May 2014 18:32:54 -0700 Subject: [PATCH 251/641] [SPARK-1594][MLLIB] Cleaning up MLlib APIs and guide Final pass before the v1.0 release. * Remove `VectorRDDs` * Move `BinaryClassificationMetrics` from `evaluation.binary` to `evaluation` * Change default value of `addIntercept` to false and allow to add intercept in Ridge and Lasso. * Clean `DecisionTree` package doc and test suite. * Mark model constructors `private[spark]` * Rename `loadLibSVMData` to `loadLibSVMFile` and hide `LabelParser` from users. * Add `saveAsLibSVMFile`. * Add `appendBias` to `MLUtils`. Author: Xiangrui Meng Closes #524 from mengxr/mllib-cleaning and squashes the following commits: 295dc8b [Xiangrui Meng] update loadLibSVMFile doc 1977ac1 [Xiangrui Meng] fix doc of appendBias 649fcf0 [Xiangrui Meng] rename loadLibSVMData to loadLibSVMFile; hide LabelParser from user APIs 54b812c [Xiangrui Meng] add appendBias a71e7d0 [Xiangrui Meng] add saveAsLibSVMFile d976295 [Xiangrui Meng] Merge branch 'master' into mllib-cleaning b7e5cec [Xiangrui Meng] remove some experimental annotations and make model constructors private[mllib] 9b02b93 [Xiangrui Meng] minor code style update a593ddc [Xiangrui Meng] fix python tests fc28c18 [Xiangrui Meng] mark more classes experimental f6cbbff [Xiangrui Meng] fix Java tests 0af70b0 [Xiangrui Meng] minor 6e139ef [Xiangrui Meng] Merge branch 'master' into mllib-cleaning 94e6dce [Xiangrui Meng] move BinaryLabelCounter and BinaryConfusionMatrixImpl to evaluation.binary df34907 [Xiangrui Meng] clean DecisionTreeSuite to use LocalSparkContext c81807f [Xiangrui Meng] set the default value of AddIntercept to false 03389c0 [Xiangrui Meng] allow to add intercept in Ridge and Lasso c66c56f [Xiangrui Meng] move tree md to package object doc a2695df [Xiangrui Meng] update guide for BinaryClassificationMetrics 9194f4c [Xiangrui Meng] move BinaryClassificationMetrics one level up 1c1a0e3 [Xiangrui Meng] remove VectorRDDs because it only contains one function that is not necessary for us to maintain --- docs/mllib-linear-methods.md | 2 +- .../examples/mllib/BinaryClassification.scala | 4 +- .../examples/mllib/LinearRegression.scala | 4 +- .../examples/mllib/SparseNaiveBayes.scala | 4 +- .../classification/LogisticRegression.scala | 7 +- .../mllib/classification/NaiveBayes.scala | 8 +- .../spark/mllib/classification/SVM.scala | 7 +- .../spark/mllib/clustering/KMeans.scala | 2 + .../spark/mllib/clustering/KMeansModel.scala | 2 +- .../BinaryClassificationMetrics.scala | 144 +++++----------- .../binary/BinaryConfusionMatrix.scala | 29 ++++ .../binary/BinaryLabelCounter.scala | 50 ++++++ .../linalg/SingularValueDecomposition.scala | 8 +- .../linalg/distributed/CoordinateMatrix.scala | 2 + .../mllib/linalg/distributed/RowMatrix.scala | 1 + .../mllib/optimization/GradientDescent.scala | 16 +- .../MatrixFactorizationModel.scala | 10 +- .../GeneralizedLinearAlgorithm.scala | 15 +- .../apache/spark/mllib/regression/Lasso.scala | 12 +- .../mllib/regression/LinearRegression.scala | 2 +- .../mllib/regression/RidgeRegression.scala | 12 +- .../org/apache/spark/mllib/tree/README.md | 17 -- .../VectorRDDs.scala => tree/package.scala} | 19 +-- .../spark/mllib/util/LabelParsers.scala | 13 +- .../org/apache/spark/mllib/util/MLUtils.scala | 155 +++++++++++++----- .../JavaLogisticRegressionSuite.java | 6 +- .../mllib/classification/JavaSVMSuite.java | 3 +- .../regression/JavaLinearRegressionSuite.java | 3 +- .../LogisticRegressionSuite.scala | 20 +-- .../spark/mllib/classification/SVMSuite.scala | 5 +- .../BinaryClassificationMetricsSuite.scala | 3 +- .../spark/mllib/rdd/VectorRDDsSuite.scala | 33 ---- .../spark/mllib/recommendation/ALSSuite.scala | 1 - .../spark/mllib/regression/LassoSuite.scala | 6 - .../regression/LinearRegressionSuite.scala | 2 +- .../regression/RidgeRegressionSuite.scala | 6 - .../spark/mllib/tree/DecisionTreeSuite.scala | 16 +- .../spark/mllib/util/MLUtilsSuite.scala | 66 ++++++-- python/pyspark/mllib/classification.py | 4 +- 39 files changed, 390 insertions(+), 329 deletions(-) rename mllib/src/main/scala/org/apache/spark/mllib/evaluation/{binary => }/BinaryClassificationMetrics.scala (64%) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryLabelCounter.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/README.md rename mllib/src/main/scala/org/apache/spark/mllib/{rdd/VectorRDDs.scala => tree/package.scala} (69%) rename mllib/src/test/scala/org/apache/spark/mllib/evaluation/{binary => }/BinaryClassificationMetricsSuite.scala (96%) delete mode 100644 mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.scala diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 8108440698495..ebb555f974bf7 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -180,7 +180,7 @@ error. {% highlight scala %} import org.apache.spark.SparkContext import org.apache.spark.mllib.classification.SVMWithSGD -import org.apache.spark.mllib.evaluation.binary.BinaryClassificationMetrics +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLUtils diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala index ec9de022c1d47..4001908c98015 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala @@ -22,7 +22,7 @@ import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.classification.{LogisticRegressionWithSGD, SVMWithSGD} -import org.apache.spark.mllib.evaluation.binary.BinaryClassificationMetrics +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.optimization.{SquaredL2Updater, L1Updater} @@ -96,7 +96,7 @@ object BinaryClassification { Logger.getRootLogger.setLevel(Level.WARN) - val examples = MLUtils.loadLibSVMData(sc, params.input).cache() + val examples = MLUtils.loadLibSVMFile(sc, params.input).cache() val splits = examples.randomSplit(Array(0.8, 0.2)) val training = splits(0).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala index 1723ca6931021..658d370f8656e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala @@ -22,7 +22,7 @@ import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.regression.LinearRegressionWithSGD -import org.apache.spark.mllib.util.{MulticlassLabelParser, MLUtils} +import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.optimization.{SimpleUpdater, SquaredL2Updater, L1Updater} /** @@ -82,7 +82,7 @@ object LinearRegression extends App { Logger.getRootLogger.setLevel(Level.WARN) - val examples = MLUtils.loadLibSVMData(sc, params.input, MulticlassLabelParser).cache() + val examples = MLUtils.loadLibSVMFile(sc, params.input, multiclass = true).cache() val splits = examples.randomSplit(Array(0.8, 0.2)) val training = splits(0).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala index 25b6768b8d72b..537e68a0991aa 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala @@ -75,8 +75,8 @@ object SparseNaiveBayes { val minPartitions = if (params.minPartitions > 0) params.minPartitions else sc.defaultMinPartitions - val examples = MLUtils.loadLibSVMData(sc, params.input, MulticlassLabelParser, - params.numFeatures, minPartitions) + val examples = + MLUtils.loadLibSVMFile(sc, params.input, multiclass = true, params.numFeatures, minPartitions) // Cache examples because it will be used in both training and evaluation. examples.cache() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 780e8bae42b84..90aa8ac998ba9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.classification +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ @@ -29,7 +30,7 @@ import org.apache.spark.rdd.RDD * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class LogisticRegressionModel( +class LogisticRegressionModel private[mllib] ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { @@ -37,18 +38,22 @@ class LogisticRegressionModel( private var threshold: Option[Double] = Some(0.5) /** + * :: Experimental :: * Sets the threshold that separates positive predictions from negative predictions. An example * with prediction score greater than or equal to this threshold is identified as an positive, * and negative otherwise. The default value is 0.5. */ + @Experimental def setThreshold(threshold: Double): this.type = { this.threshold = Some(threshold) this } /** + * :: Experimental :: * Clears the threshold so that `predict` will output raw prediction scores. */ + @Experimental def clearThreshold(): this.type = { threshold = None this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index f6f62ce2de04e..b6e0c4a80e27b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -19,7 +19,6 @@ package org.apache.spark.mllib.classification import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} -import org.apache.spark.annotation.Experimental import org.apache.spark.Logging import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vector @@ -27,7 +26,6 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD /** - * :: Experimental :: * Model for Naive Bayes Classifiers. * * @param labels list of labels @@ -35,8 +33,7 @@ import org.apache.spark.rdd.RDD * @param theta log of class conditional probabilities, whose dimension is C-by-D, * where D is number of features */ -@Experimental -class NaiveBayesModel( +class NaiveBayesModel private[mllib] ( val labels: Array[Double], val pi: Array[Double], val theta: Array[Array[Double]]) extends ClassificationModel with Serializable { @@ -124,6 +121,9 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with } } +/** + * Top-level methods for calling naive Bayes. + */ object NaiveBayes { /** * Trains a Naive Bayes model given an RDD of `(label, features)` pairs. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 81b126717e9a4..e05213536e64a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.classification +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ @@ -29,7 +30,7 @@ import org.apache.spark.rdd.RDD * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class SVMModel( +class SVMModel private[mllib] ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { @@ -37,18 +38,22 @@ class SVMModel( private var threshold: Option[Double] = Some(0.0) /** + * :: Experimental :: * Sets the threshold that separates positive predictions from negative predictions. An example * with prediction score greater than or equal to this threshold is identified as an positive, * and negative otherwise. The default value is 0.0. */ + @Experimental def setThreshold(threshold: Double): this.type = { this.threshold = Some(threshold) this } /** + * :: Experimental :: * Clears the threshold so that `predict` will output raw prediction scores. */ + @Experimental def clearThreshold(): this.type = { threshold = None this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index a64c5d44be406..de22fbb6ffc10 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{DenseVector => BDV, Vector => BV, norm => breezeNorm} +import org.apache.spark.annotation.Experimental import org.apache.spark.Logging import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.{Vector, Vectors} @@ -81,6 +82,7 @@ class KMeans private ( * this many times with random starting conditions (configured by the initialization mode), then * return the best clustering found over any run. Default: 1. */ + @Experimental def setRuns(runs: Int): KMeans = { if (runs <= 0) { throw new IllegalArgumentException("Number of runs must be positive") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 18abbf2758b86..ce14b06241932 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -24,7 +24,7 @@ import org.apache.spark.mllib.linalg.Vector /** * A clustering model for K-means. Each point belongs to the cluster with the closest center. */ -class KMeansModel(val clusterCenters: Array[Vector]) extends Serializable { +class KMeansModel private[mllib] (val clusterCenters: Array[Vector]) extends Serializable { /** Total number of clusters. */ def k: Int = clusterCenters.length diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala similarity index 64% rename from mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala rename to mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala index ed7b0fc943367..079743742d86d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala @@ -15,83 +15,22 @@ * limitations under the License. */ -package org.apache.spark.mllib.evaluation.binary +package org.apache.spark.mllib.evaluation -import org.apache.spark.rdd.{UnionRDD, RDD} -import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.evaluation.AreaUnderCurve +import org.apache.spark.annotation.Experimental import org.apache.spark.Logging +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.evaluation.binary._ +import org.apache.spark.rdd.{RDD, UnionRDD} /** - * Implementation of [[org.apache.spark.mllib.evaluation.binary.BinaryConfusionMatrix]]. - * - * @param count label counter for labels with scores greater than or equal to the current score - * @param totalCount label counter for all labels - */ -private case class BinaryConfusionMatrixImpl( - count: LabelCounter, - totalCount: LabelCounter) extends BinaryConfusionMatrix with Serializable { - - /** number of true positives */ - override def numTruePositives: Long = count.numPositives - - /** number of false positives */ - override def numFalsePositives: Long = count.numNegatives - - /** number of false negatives */ - override def numFalseNegatives: Long = totalCount.numPositives - count.numPositives - - /** number of true negatives */ - override def numTrueNegatives: Long = totalCount.numNegatives - count.numNegatives - - /** number of positives */ - override def numPositives: Long = totalCount.numPositives - - /** number of negatives */ - override def numNegatives: Long = totalCount.numNegatives -} - -/** + * :: Experimental :: * Evaluator for binary classification. * * @param scoreAndLabels an RDD of (score, label) pairs. */ -class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) - extends Serializable with Logging { - - private lazy val ( - cumulativeCounts: RDD[(Double, LabelCounter)], - confusions: RDD[(Double, BinaryConfusionMatrix)]) = { - // Create a bin for each distinct score value, count positives and negatives within each bin, - // and then sort by score values in descending order. - val counts = scoreAndLabels.combineByKey( - createCombiner = (label: Double) => new LabelCounter(0L, 0L) += label, - mergeValue = (c: LabelCounter, label: Double) => c += label, - mergeCombiners = (c1: LabelCounter, c2: LabelCounter) => c1 += c2 - ).sortByKey(ascending = false) - val agg = counts.values.mapPartitions({ iter => - val agg = new LabelCounter() - iter.foreach(agg += _) - Iterator(agg) - }, preservesPartitioning = true).collect() - val partitionwiseCumulativeCounts = - agg.scanLeft(new LabelCounter())((agg: LabelCounter, c: LabelCounter) => agg.clone() += c) - val totalCount = partitionwiseCumulativeCounts.last - logInfo(s"Total counts: $totalCount") - val cumulativeCounts = counts.mapPartitionsWithIndex( - (index: Int, iter: Iterator[(Double, LabelCounter)]) => { - val cumCount = partitionwiseCumulativeCounts(index) - iter.map { case (score, c) => - cumCount += c - (score, cumCount.clone()) - } - }, preservesPartitioning = true) - cumulativeCounts.persist() - val confusions = cumulativeCounts.map { case (score, cumCount) => - (score, BinaryConfusionMatrixImpl(cumCount, totalCount).asInstanceOf[BinaryConfusionMatrix]) - } - (cumulativeCounts, confusions) - } +@Experimental +class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends Logging { /** Unpersist intermediate RDDs used in the computation. */ def unpersist() { @@ -154,6 +93,41 @@ class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) /** Returns the (threshold, recall) curve. */ def recallByThreshold(): RDD[(Double, Double)] = createCurve(Recall) + private lazy val ( + cumulativeCounts: RDD[(Double, BinaryLabelCounter)], + confusions: RDD[(Double, BinaryConfusionMatrix)]) = { + // Create a bin for each distinct score value, count positives and negatives within each bin, + // and then sort by score values in descending order. + val counts = scoreAndLabels.combineByKey( + createCombiner = (label: Double) => new BinaryLabelCounter(0L, 0L) += label, + mergeValue = (c: BinaryLabelCounter, label: Double) => c += label, + mergeCombiners = (c1: BinaryLabelCounter, c2: BinaryLabelCounter) => c1 += c2 + ).sortByKey(ascending = false) + val agg = counts.values.mapPartitions({ iter => + val agg = new BinaryLabelCounter() + iter.foreach(agg += _) + Iterator(agg) + }, preservesPartitioning = true).collect() + val partitionwiseCumulativeCounts = + agg.scanLeft(new BinaryLabelCounter())( + (agg: BinaryLabelCounter, c: BinaryLabelCounter) => agg.clone() += c) + val totalCount = partitionwiseCumulativeCounts.last + logInfo(s"Total counts: $totalCount") + val cumulativeCounts = counts.mapPartitionsWithIndex( + (index: Int, iter: Iterator[(Double, BinaryLabelCounter)]) => { + val cumCount = partitionwiseCumulativeCounts(index) + iter.map { case (score, c) => + cumCount += c + (score, cumCount.clone()) + } + }, preservesPartitioning = true) + cumulativeCounts.persist() + val confusions = cumulativeCounts.map { case (score, cumCount) => + (score, BinaryConfusionMatrixImpl(cumCount, totalCount).asInstanceOf[BinaryConfusionMatrix]) + } + (cumulativeCounts, confusions) + } + /** Creates a curve of (threshold, metric). */ private def createCurve(y: BinaryClassificationMetricComputer): RDD[(Double, Double)] = { confusions.map { case (s, c) => @@ -170,35 +144,3 @@ class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) } } } - -/** - * A counter for positives and negatives. - * - * @param numPositives number of positive labels - * @param numNegatives number of negative labels - */ -private class LabelCounter( - var numPositives: Long = 0L, - var numNegatives: Long = 0L) extends Serializable { - - /** Processes a label. */ - def +=(label: Double): LabelCounter = { - // Though we assume 1.0 for positive and 0.0 for negative, the following check will handle - // -1.0 for negative as well. - if (label > 0.5) numPositives += 1L else numNegatives += 1L - this - } - - /** Merges another counter. */ - def +=(other: LabelCounter): LabelCounter = { - numPositives += other.numPositives - numNegatives += other.numNegatives - this - } - - override def clone: LabelCounter = { - new LabelCounter(numPositives, numNegatives) - } - - override def toString: String = s"{numPos: $numPositives, numNeg: $numNegatives}" -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala index 75a75b216002a..559c6ef7e7251 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala @@ -39,3 +39,32 @@ private[evaluation] trait BinaryConfusionMatrix { /** number of negatives */ def numNegatives: Long = numFalsePositives + numTrueNegatives } + +/** + * Implementation of [[org.apache.spark.mllib.evaluation.binary.BinaryConfusionMatrix]]. + * + * @param count label counter for labels with scores greater than or equal to the current score + * @param totalCount label counter for all labels + */ +private[evaluation] case class BinaryConfusionMatrixImpl( + count: BinaryLabelCounter, + totalCount: BinaryLabelCounter) extends BinaryConfusionMatrix { + + /** number of true positives */ + override def numTruePositives: Long = count.numPositives + + /** number of false positives */ + override def numFalsePositives: Long = count.numNegatives + + /** number of false negatives */ + override def numFalseNegatives: Long = totalCount.numPositives - count.numPositives + + /** number of true negatives */ + override def numTrueNegatives: Long = totalCount.numNegatives - count.numNegatives + + /** number of positives */ + override def numPositives: Long = totalCount.numPositives + + /** number of negatives */ + override def numNegatives: Long = totalCount.numNegatives +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryLabelCounter.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryLabelCounter.scala new file mode 100644 index 0000000000000..1e610c20092a7 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryLabelCounter.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation.binary + +/** + * A counter for positives and negatives. + * + * @param numPositives number of positive labels + * @param numNegatives number of negative labels + */ +private[evaluation] class BinaryLabelCounter( + var numPositives: Long = 0L, + var numNegatives: Long = 0L) extends Serializable { + + /** Processes a label. */ + def +=(label: Double): BinaryLabelCounter = { + // Though we assume 1.0 for positive and 0.0 for negative, the following check will handle + // -1.0 for negative as well. + if (label > 0.5) numPositives += 1L else numNegatives += 1L + this + } + + /** Merges another counter. */ + def +=(other: BinaryLabelCounter): BinaryLabelCounter = { + numPositives += other.numPositives + numNegatives += other.numNegatives + this + } + + override def clone: BinaryLabelCounter = { + new BinaryLabelCounter(numPositives, numNegatives) + } + + override def toString: String = s"{numPos: $numPositives, numNeg: $numNegatives}" +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala index 46b105457430c..9669c364bad8f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala @@ -17,5 +17,11 @@ package org.apache.spark.mllib.linalg -/** Represents singular value decomposition (SVD) factors. */ +import org.apache.spark.annotation.Experimental + +/** + * :: Experimental :: + * Represents singular value decomposition (SVD) factors. + */ +@Experimental case class SingularValueDecomposition[UType, VType](U: UType, s: Vector, V: VType) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala index 56b8fdcda66eb..06d8915f3bfa1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -25,11 +25,13 @@ import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vectors /** + * :: Experimental :: * Represents an entry in an distributed matrix. * @param i row index * @param j column index * @param value value of the entry */ +@Experimental case class MatrixEntry(i: Long, j: Long, value: Double) /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 0c0afcd9ec0d7..b10857fe7c8a1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -427,6 +427,7 @@ class RowMatrix( } } +@Experimental object RowMatrix { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index c75909bac9248..7030eeabe400a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -21,19 +21,17 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{DenseVector => BDV} -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{Experimental, DeveloperApi} import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.{Vectors, Vector} /** - * :: DeveloperApi :: * Class used to solve an optimization problem using Gradient Descent. * @param gradient Gradient function to be used. * @param updater Updater to be used to update weights after every iteration. */ -@DeveloperApi -class GradientDescent(private var gradient: Gradient, private var updater: Updater) +class GradientDescent private[mllib] (private var gradient: Gradient, private var updater: Updater) extends Optimizer with Logging { private var stepSize: Double = 1.0 @@ -51,9 +49,11 @@ class GradientDescent(private var gradient: Gradient, private var updater: Updat } /** + * :: Experimental :: * Set fraction of data to be used for each SGD iteration. * Default 1.0 (corresponding to deterministic/classical gradient descent) */ + @Experimental def setMiniBatchFraction(fraction: Double): this.type = { this.miniBatchFraction = fraction this @@ -95,6 +95,14 @@ class GradientDescent(private var gradient: Gradient, private var updater: Updat this } + /** + * :: DeveloperApi :: + * Runs gradient descent on the given training data. + * @param data training data + * @param initialWeights initial weights + * @return solution vector + */ + @DeveloperApi def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector = { val (weights, _) = GradientDescent.runMiniBatchSGD( data, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 471546cd82c7d..899286d235a9d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.recommendation -import org.jblas._ +import org.jblas.DoubleMatrix import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.JavaRDD @@ -25,7 +25,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.mllib.api.python.PythonMLLibAPI - /** * Model representing the result of matrix factorization. * @@ -35,12 +34,10 @@ import org.apache.spark.mllib.api.python.PythonMLLibAPI * @param productFeatures RDD of tuples where each tuple represents the productId * and the features computed for this product. */ -class MatrixFactorizationModel( +class MatrixFactorizationModel private[mllib] ( val rank: Int, val userFeatures: RDD[(Int, Array[Double])], - val productFeatures: RDD[(Int, Array[Double])]) - extends Serializable -{ + val productFeatures: RDD[(Int, Array[Double])]) extends Serializable { /** Predict the rating of one user for one product. */ def predict(user: Int, product: Int): Double = { val userVector = new DoubleMatrix(userFeatures.lookup(user).head) @@ -76,6 +73,7 @@ class MatrixFactorizationModel( * @param usersProductsJRDD A JavaRDD with serialized tuples (user, product) * @return JavaRDD of serialized Rating objects. */ + @DeveloperApi def predict(usersProductsJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = { val pythonAPI = new PythonMLLibAPI() val usersProducts = usersProductsJRDD.rdd.map(xBytes => pythonAPI.unpackTuple(xBytes)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index d969e7aa60061..8cca926f1c92e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -19,13 +19,14 @@ package org.apache.spark.mllib.regression import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg.{Vectors, Vector} /** + * :: DeveloperApi :: * GeneralizedLinearModel (GLM) represents a model trained using * GeneralizedLinearAlgorithm. GLMs consist of a weight vector and * an intercept. @@ -33,6 +34,7 @@ import org.apache.spark.mllib.linalg.{Vectors, Vector} * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ +@DeveloperApi abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double) extends Serializable { @@ -72,9 +74,11 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double } /** + * :: DeveloperApi :: * GeneralizedLinearAlgorithm implements methods to train a Generalized Linear Model (GLM). * This class should be extended with an Optimizer to create a new GLM. */ +@DeveloperApi abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] extends Logging with Serializable { @@ -83,8 +87,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] /** The optimizer to solve the problem. */ def optimizer: Optimizer - /** Whether to add intercept (default: true). */ - protected var addIntercept: Boolean = true + /** Whether to add intercept (default: false). */ + protected var addIntercept: Boolean = false protected var validateData: Boolean = true @@ -94,7 +98,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] protected def createModel(weights: Vector, intercept: Double): M /** - * Set if the algorithm should add an intercept. Default true. + * Set if the algorithm should add an intercept. Default false. + * We set the default to false because adding the intercept will cause memory allocation. */ def setIntercept(addIntercept: Boolean): this.type = { this.addIntercept = addIntercept @@ -102,10 +107,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] } /** - * :: Experimental :: * Set if the algorithm should validate data before training. Default true. */ - @Experimental def setValidateData(validateData: Boolean): this.type = { this.validateData = validateData this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index 0e6fb1b1caa41..a05dfc045fb8e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.regression +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.rdd.RDD @@ -27,7 +28,7 @@ import org.apache.spark.rdd.RDD * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class LassoModel( +class LassoModel private[mllib] ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) @@ -64,21 +65,12 @@ class LassoWithSGD private ( .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) - // We don't want to penalize the intercept, so set this to false. - super.setIntercept(false) - /** * Construct a Lasso object with default parameters: {stepSize: 1.0, numIterations: 100, * regParam: 1.0, miniBatchFraction: 1.0}. */ def this() = this(1.0, 100, 1.0, 1.0) - override def setIntercept(addIntercept: Boolean): this.type = { - // TODO: Support adding intercept. - if (addIntercept) throw new UnsupportedOperationException("Adding intercept is not supported.") - this - } - override protected def createModel(weights: Vector, intercept: Double) = { new LassoModel(weights, intercept) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 1532ff90d846d..0ebad4eb58d88 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -27,7 +27,7 @@ import org.apache.spark.mllib.optimization._ * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class LinearRegressionModel( +class LinearRegressionModel private[mllib] ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index 5f7e25a9b8be1..bd983bac001a0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.regression +import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg.Vector @@ -27,7 +28,7 @@ import org.apache.spark.mllib.linalg.Vector * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class RidgeRegressionModel( +class RidgeRegressionModel private[mllib] ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) @@ -65,21 +66,12 @@ class RidgeRegressionWithSGD private ( .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) - // We don't want to penalize the intercept in RidgeRegression, so set this to false. - super.setIntercept(false) - /** * Construct a RidgeRegression object with default parameters: {stepSize: 1.0, numIterations: 100, * regParam: 1.0, miniBatchFraction: 1.0}. */ def this() = this(1.0, 100, 1.0, 1.0) - override def setIntercept(addIntercept: Boolean): this.type = { - // TODO: Support adding intercept. - if (addIntercept) throw new UnsupportedOperationException("Adding intercept is not supported.") - this - } - override protected def createModel(weights: Vector, intercept: Double) = { new RidgeRegressionModel(weights, intercept) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/README.md b/mllib/src/main/scala/org/apache/spark/mllib/tree/README.md deleted file mode 100644 index 0fd71aa9735bc..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/README.md +++ /dev/null @@ -1,17 +0,0 @@ -This package contains the default implementation of the decision tree algorithm. - -The decision tree algorithm supports: -+ Binary classification -+ Regression -+ Information loss calculation with entropy and gini for classification and variance for regression -+ Both continuous and categorical features - -# Tree improvements -+ Node model pruning -+ Printing to dot files - -# Future Ensemble Extensions - -+ Random forests -+ Boosting -+ Extremely randomized trees diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/package.scala similarity index 69% rename from mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala rename to mllib/src/main/scala/org/apache/spark/mllib/tree/package.scala index 9096d6a1a16d6..bcaacc1b1f191 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/package.scala @@ -15,18 +15,15 @@ * limitations under the License. */ -package org.apache.spark.mllib.rdd - -import org.apache.spark.rdd.RDD -import org.apache.spark.mllib.linalg.{Vectors, Vector} +package org.apache.spark.mllib /** - * Factory methods for `RDD[Vector]`. + * This package contains the default implementation of the decision tree algorithm, which supports: + * - binary classification, + * - regression, + * - information loss calculation with entropy and Gini for classification and + * variance for regression, + * - both continuous and categorical features. */ -object VectorRDDs { - - /** - * Converts an `RDD[Array[Double]]` to `RDD[Vector]`. - */ - def fromArrayRDD(rdd: RDD[Array[Double]]): RDD[Vector] = rdd.map(v => Vectors.dense(v)) +package object tree { } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala index f7966d3ebb613..e25bf18b780bf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala @@ -18,16 +18,23 @@ package org.apache.spark.mllib.util /** Trait for label parsers. */ -trait LabelParser extends Serializable { +private trait LabelParser extends Serializable { /** Parses a string label into a double label. */ def parse(labelString: String): Double } +/** Factory methods for label parsers. */ +private object LabelParser { + def getInstance(multiclass: Boolean): LabelParser = { + if (multiclass) MulticlassLabelParser else BinaryLabelParser + } +} + /** * Label parser for binary labels, which outputs 1.0 (positive) if the value is greater than 0.5, * or 0.0 (negative) otherwise. So it works with +1/-1 labeling and +1/0 labeling. */ -object BinaryLabelParser extends LabelParser { +private object BinaryLabelParser extends LabelParser { /** Gets the default instance of BinaryLabelParser. */ def getInstance(): LabelParser = this @@ -41,7 +48,7 @@ object BinaryLabelParser extends LabelParser { /** * Label parser for multiclass labels, which converts the input label to double. */ -object MulticlassLabelParser extends LabelParser { +private object MulticlassLabelParser extends LabelParser { /** Gets the default instance of MulticlassLabelParser. */ def getInstance(): LabelParser = this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 3d6e7e0d5c953..e598b6cb171a8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -19,16 +19,17 @@ package org.apache.spark.mllib.util import scala.reflect.ClassTag -import breeze.linalg.{Vector => BV, SparseVector => BSV, squaredDistance => breezeSquaredDistance} +import breeze.linalg.{Vector => BV, DenseVector => BDV, SparseVector => BSV, + squaredDistance => breezeSquaredDistance} import org.apache.spark.annotation.Experimental import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.rdd.PartitionwiseSampledRDD -import org.apache.spark.SparkContext._ import org.apache.spark.util.random.BernoulliSampler import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.storage.StorageLevel /** * Helper methods to load, save and pre-process data used in ML Lib. @@ -54,13 +55,16 @@ object MLUtils { * * @param sc Spark context * @param path file or directory path in any Hadoop-supported file system URI - * @param labelParser parser for labels, default: 1.0 if label > 0.5 or 0.0 otherwise + * @param labelParser parser for labels * @param numFeatures number of features, which will be determined from the input data if a - * negative value is given. The default value is -1. - * @param minPartitions min number of partitions, default: sc.defaultMinPartitions + * nonpositive value is given. This is useful when the dataset is already split + * into multiple files and you want to load them separately, because some + * features may not present in certain files, which leads to inconsistent + * feature dimensions. + * @param minPartitions min number of partitions * @return labeled data stored as an RDD[LabeledPoint] */ - def loadLibSVMData( + private def loadLibSVMFile( sc: SparkContext, path: String, labelParser: LabelParser, @@ -68,63 +72,112 @@ object MLUtils { minPartitions: Int): RDD[LabeledPoint] = { val parsed = sc.textFile(path, minPartitions) .map(_.trim) - .filter(!_.isEmpty) - .map(_.split(' ')) + .filter(line => !(line.isEmpty || line.startsWith("#"))) + .map { line => + val items = line.split(' ') + val label = labelParser.parse(items.head) + val (indices, values) = items.tail.map { item => + val indexAndValue = item.split(':') + val index = indexAndValue(0).toInt - 1 // Convert 1-based indices to 0-based. + val value = indexAndValue(1).toDouble + (index, value) + }.unzip + (label, indices.toArray, values.toArray) + } + // Determine number of features. - val d = if (numFeatures >= 0) { + val d = if (numFeatures > 0) { numFeatures } else { - parsed.map { items => - if (items.length > 1) { - items.last.split(':')(0).toInt - } else { - 0 - } - }.reduce(math.max) + parsed.persist(StorageLevel.MEMORY_ONLY) + parsed.map { case (label, indices, values) => + indices.lastOption.getOrElse(0) + }.reduce(math.max) + 1 } - parsed.map { items => - val label = labelParser.parse(items.head) - val (indices, values) = items.tail.map { item => - val indexAndValue = item.split(':') - val index = indexAndValue(0).toInt - 1 - val value = indexAndValue(1).toDouble - (index, value) - }.unzip - LabeledPoint(label, Vectors.sparse(d, indices.toArray, values.toArray)) + + parsed.map { case (label, indices, values) => + LabeledPoint(label, Vectors.sparse(d, indices, values)) } } - // Convenient methods for calling from Java. + // Convenient methods for `loadLibSVMFile`. /** - * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint], - * with number of features determined automatically and the default number of partitions. + * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint]. + * The LIBSVM format is a text-based format used by LIBSVM and LIBLINEAR. + * Each line represents a labeled sparse feature vector using the following format: + * {{{label index1:value1 index2:value2 ...}}} + * where the indices are one-based and in ascending order. + * This method parses each line into a [[org.apache.spark.mllib.regression.LabeledPoint]], + * where the feature indices are converted to zero-based. + * + * @param sc Spark context + * @param path file or directory path in any Hadoop-supported file system URI + * @param multiclass whether the input labels contain more than two classes. If false, any label + * with value greater than 0.5 will be mapped to 1.0, or 0.0 otherwise. So it + * works for both +1/-1 and 1/0 cases. If true, the double value parsed directly + * from the label string will be used as the label value. + * @param numFeatures number of features, which will be determined from the input data if a + * nonpositive value is given. This is useful when the dataset is already split + * into multiple files and you want to load them separately, because some + * features may not present in certain files, which leads to inconsistent + * feature dimensions. + * @param minPartitions min number of partitions + * @return labeled data stored as an RDD[LabeledPoint] */ - def loadLibSVMData(sc: SparkContext, path: String): RDD[LabeledPoint] = - loadLibSVMData(sc, path, BinaryLabelParser, -1, sc.defaultMinPartitions) + def loadLibSVMFile( + sc: SparkContext, + path: String, + multiclass: Boolean, + numFeatures: Int, + minPartitions: Int): RDD[LabeledPoint] = + loadLibSVMFile(sc, path, LabelParser.getInstance(multiclass), numFeatures, minPartitions) /** - * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], - * with the given label parser, number of features determined automatically, - * and the default number of partitions. + * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], with the default number of + * partitions. */ - def loadLibSVMData( + def loadLibSVMFile( sc: SparkContext, path: String, - labelParser: LabelParser): RDD[LabeledPoint] = - loadLibSVMData(sc, path, labelParser, -1, sc.defaultMinPartitions) + multiclass: Boolean, + numFeatures: Int): RDD[LabeledPoint] = + loadLibSVMFile(sc, path, multiclass, numFeatures, sc.defaultMinPartitions) /** - * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], - * with the given label parser, number of features specified explicitly, - * and the default number of partitions. + * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], with the number of features + * determined automatically and the default number of partitions. */ - def loadLibSVMData( + def loadLibSVMFile( sc: SparkContext, path: String, - labelParser: LabelParser, - numFeatures: Int): RDD[LabeledPoint] = - loadLibSVMData(sc, path, labelParser, numFeatures, sc.defaultMinPartitions) + multiclass: Boolean): RDD[LabeledPoint] = + loadLibSVMFile(sc, path, multiclass, -1, sc.defaultMinPartitions) + + /** + * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint], with number of + * features determined automatically and the default number of partitions. + */ + def loadLibSVMFile(sc: SparkContext, path: String): RDD[LabeledPoint] = + loadLibSVMFile(sc, path, multiclass = false, -1, sc.defaultMinPartitions) + + /** + * Save labeled data in LIBSVM format. + * @param data an RDD of LabeledPoint to be saved + * @param dir directory to save the data + * + * @see [[org.apache.spark.mllib.util.MLUtils#loadLibSVMFile]] + */ + def saveAsLibSVMFile(data: RDD[LabeledPoint], dir: String) { + // TODO: allow to specify label precision and feature precision. + val dataStr = data.map { case LabeledPoint(label, features) => + val featureStrings = features.toBreeze.activeIterator.map { case (i, v) => + s"${i + 1}:$v" + } + (Iterator(label) ++ featureStrings).mkString(" ") + } + dataStr.saveAsTextFile(dir) + } /** * :: Experimental :: @@ -163,10 +216,12 @@ object MLUtils { } /** + * :: Experimental :: * Return a k element array of pairs of RDDs with the first element of each pair * containing the training data, a complement of the validation data and the second * element, the validation data, containing a unique 1/kth of the data. Where k=numFolds. */ + @Experimental def kFold[T: ClassTag](rdd: RDD[T], numFolds: Int, seed: Int): Array[(RDD[T], RDD[T])] = { val numFoldsF = numFolds.toFloat (1 to numFolds).map { fold => @@ -178,6 +233,18 @@ object MLUtils { }.toArray } + /** + * Returns a new vector with `1.0` (bias) appended to the input vector. + */ + def appendBias(vector: Vector): Vector = { + val vector1 = vector.toBreeze match { + case dv: BDV[Double] => BDV.vertcat(dv, new BDV[Double](Array(1.0))) + case sv: BSV[Double] => BSV.vertcat(sv, new BSV[Double](Array(0), Array(1.0), 1)) + case v: Any => throw new IllegalArgumentException("Do not support vector type " + v.getClass) + } + Vectors.fromBreeze(vector1) + } + /** * Returns the squared Euclidean distance between two vectors. The following formula will be used * if it does not introduce too much numerical error: diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java index e18e3bc6a86be..d75d3a6b26730 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java @@ -68,6 +68,7 @@ public void runLRUsingConstructor() { LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17); LogisticRegressionWithSGD lrImpl = new LogisticRegressionWithSGD(); + lrImpl.setIntercept(true); lrImpl.optimizer().setStepSize(1.0) .setRegParam(1.0) .setNumIterations(100); @@ -80,8 +81,8 @@ public void runLRUsingConstructor() { @Test public void runLRUsingStaticMethods() { int nPoints = 10000; - double A = 2.0; - double B = -1.5; + double A = 0.0; + double B = -2.5; JavaRDD testRDD = sc.parallelize( LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); @@ -92,6 +93,7 @@ public void runLRUsingStaticMethods() { testRDD.rdd(), 100, 1.0, 1.0); int numAccurate = validatePrediction(validationData, model); + System.out.println(numAccurate); Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); } diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java index 4701a5e545020..667f76a1bd55f 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java @@ -67,6 +67,7 @@ public void runSVMUsingConstructor() { SVMSuite.generateSVMInputAsList(A, weights, nPoints, 17); SVMWithSGD svmSGDImpl = new SVMWithSGD(); + svmSGDImpl.setIntercept(true); svmSGDImpl.optimizer().setStepSize(1.0) .setRegParam(1.0) .setNumIterations(100); @@ -79,7 +80,7 @@ public void runSVMUsingConstructor() { @Test public void runSVMUsingStaticMethods() { int nPoints = 10000; - double A = 2.0; + double A = 0.0; double[] weights = {-1.5, 1.0}; JavaRDD testRDD = sc.parallelize(SVMSuite.generateSVMInputAsList(A, diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java index 5a4410a632649..7151e553512b3 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java @@ -68,6 +68,7 @@ public void runLinearRegressionUsingConstructor() { LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(); + linSGDImpl.setIntercept(true); LinearRegressionModel model = linSGDImpl.run(testRDD.rdd()); int numAccurate = validatePrediction(validationData, model); @@ -77,7 +78,7 @@ public void runLinearRegressionUsingConstructor() { @Test public void runLinearRegressionUsingStaticMethods() { int nPoints = 100; - double A = 3.0; + double A = 0.0; double[] weights = {10, 10}; JavaRDD testRDD = sc.parallelize( diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 1e03c9df820b0..4d7b984e3ec29 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -46,24 +46,14 @@ object LogisticRegressionSuite { val rnd = new Random(seed) val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - // NOTE: if U is uniform[0, 1] then ln(u) - ln(1-u) is Logistic(0,1) - val unifRand = new scala.util.Random(45) - val rLogis = (0 until nPoints).map { i => - val u = unifRand.nextDouble() - math.log(u) - math.log(1.0-u) - } - - // y <- A + B*x + rLogis() - // y <- as.numeric(y > 0) - val y: Seq[Int] = (0 until nPoints).map { i => - val yVal = offset + scale * x1(i) + rLogis(i) - if (yVal > 0) 1 else 0 + val y = (0 until nPoints).map { i => + val p = 1.0 / (1.0 + math.exp(-(offset + scale * x1(i)))) + if (rnd.nextDouble() < p) 1.0 else 0.0 } val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(Array(x1(i))))) testData } - } class LogisticRegressionSuite extends FunSuite with LocalSparkContext with ShouldMatchers { @@ -85,7 +75,7 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Shoul val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val lr = new LogisticRegressionWithSGD() + val lr = new LogisticRegressionWithSGD().setIntercept(true) lr.optimizer.setStepSize(10.0).setNumIterations(20) val model = lr.run(testRDD) @@ -118,7 +108,7 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Shoul testRDD.cache() // Use half as many iterations as the previous test. - val lr = new LogisticRegressionWithSGD() + val lr = new LogisticRegressionWithSGD().setIntercept(true) lr.optimizer.setStepSize(10.0).setNumIterations(10) val model = lr.run(testRDD, initialWeights) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala index dfacbfeee6fb4..77d6f04b32320 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala @@ -69,7 +69,6 @@ class SVMSuite extends FunSuite with LocalSparkContext { assert(numOffPredictions < input.length / 5) } - test("SVM using local random SGD") { val nPoints = 10000 @@ -83,7 +82,7 @@ class SVMSuite extends FunSuite with LocalSparkContext { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val svm = new SVMWithSGD() + val svm = new SVMWithSGD().setIntercept(true) svm.optimizer.setStepSize(1.0).setRegParam(1.0).setNumIterations(100) val model = svm.run(testRDD) @@ -115,7 +114,7 @@ class SVMSuite extends FunSuite with LocalSparkContext { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val svm = new SVMWithSGD() + val svm = new SVMWithSGD().setIntercept(true) svm.optimizer.setStepSize(1.0).setRegParam(1.0).setNumIterations(100) val model = svm.run(testRDD, initialWeights) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala similarity index 96% rename from mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala index 173fdaefab3da..9d16182f9d8c4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.spark.mllib.evaluation.binary +package org.apache.spark.mllib.evaluation import org.scalatest.FunSuite import org.apache.spark.mllib.util.LocalSparkContext -import org.apache.spark.mllib.evaluation.AreaUnderCurve class BinaryClassificationMetricsSuite extends FunSuite with LocalSparkContext { test("binary evaluation metrics") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.scala deleted file mode 100644 index 692f025e959ae..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.scala +++ /dev/null @@ -1,33 +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.mllib.rdd - -import org.scalatest.FunSuite - -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.LocalSparkContext - -class VectorRDDsSuite extends FunSuite with LocalSparkContext { - - test("from array rdd") { - val data = Seq(Array(1.0, 2.0), Array(3.0, 4.0)) - val arrayRdd = sc.parallelize(data, 2) - val vectorRdd = VectorRDDs.fromArrayRDD(arrayRdd) - assert(arrayRdd.collect().map(v => Vectors.dense(v)) === vectorRdd.collect()) - } -} 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 4dfcd4b52ec66..2d944f3eb7ff9 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 @@ -27,7 +27,6 @@ import org.jblas.DoubleMatrix import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.Partitioner object ALSSuite { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala index 6aad9eb84e13c..bfa42959c8ead 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala @@ -112,10 +112,4 @@ class LassoSuite extends FunSuite with LocalSparkContext { // Test prediction on Array. validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } - - test("do not support intercept") { - intercept[UnsupportedOperationException] { - new LassoWithSGD().setIntercept(true) - } - } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala index 2f7d30708ce17..7aaad7d7a3e39 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala @@ -37,7 +37,7 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { test("linear regression") { val testRDD = sc.parallelize(LinearDataGenerator.generateLinearInput( 3.0, Array(10.0, 10.0), 100, 42), 2).cache() - val linReg = new LinearRegressionWithSGD() + val linReg = new LinearRegressionWithSGD().setIntercept(true) linReg.optimizer.setNumIterations(1000).setStepSize(1.0) val model = linReg.run(testRDD) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala index f66fc6ea6c1ec..67768e17fbe6d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala @@ -72,10 +72,4 @@ class RidgeRegressionSuite extends FunSuite with LocalSparkContext { assert(ridgeErr < linearErr, "ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")") } - - test("do not support intercept") { - intercept[UnsupportedOperationException] { - new RidgeRegressionWithSGD().setIntercept(true) - } - } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 350130c914f26..be383aab714d3 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -17,10 +17,8 @@ package org.apache.spark.mllib.tree -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite -import org.apache.spark.SparkContext import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} import org.apache.spark.mllib.tree.model.Filter @@ -28,19 +26,9 @@ import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.LocalSparkContext -class DecisionTreeSuite extends FunSuite with BeforeAndAfterAll { - - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } +class DecisionTreeSuite extends FunSuite with LocalSparkContext { test("split and bin calculation") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 674378a34ce34..3f64baf6fe41f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -19,8 +19,8 @@ package org.apache.spark.mllib.util import java.io.File +import scala.io.Source import scala.math -import scala.util.Random import org.scalatest.FunSuite @@ -29,7 +29,8 @@ import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, norm => breezeNor import com.google.common.base.Charsets import com.google.common.io.Files -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils._ class MLUtilsSuite extends FunSuite with LocalSparkContext { @@ -58,7 +59,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { } } - test("loadLibSVMData") { + test("loadLibSVMFile") { val lines = """ |+1 1:1.0 3:2.0 5:3.0 @@ -70,8 +71,8 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { Files.write(lines, file, Charsets.US_ASCII) val path = tempDir.toURI.toString - val pointsWithNumFeatures = MLUtils.loadLibSVMData(sc, path, BinaryLabelParser, 6).collect() - val pointsWithoutNumFeatures = MLUtils.loadLibSVMData(sc, path).collect() + val pointsWithNumFeatures = loadLibSVMFile(sc, path, multiclass = false, 6).collect() + val pointsWithoutNumFeatures = loadLibSVMFile(sc, path).collect() for (points <- Seq(pointsWithNumFeatures, pointsWithoutNumFeatures)) { assert(points.length === 3) @@ -83,29 +84,54 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { assert(points(2).features === Vectors.sparse(6, Seq((1, 4.0), (3, 5.0), (5, 6.0)))) } - val multiclassPoints = MLUtils.loadLibSVMData(sc, path, MulticlassLabelParser).collect() + val multiclassPoints = loadLibSVMFile(sc, path, multiclass = true).collect() assert(multiclassPoints.length === 3) assert(multiclassPoints(0).label === 1.0) assert(multiclassPoints(1).label === -1.0) assert(multiclassPoints(2).label === -1.0) - try { - file.delete() - tempDir.delete() - } catch { - case t: Throwable => - } + deleteQuietly(tempDir) + } + + test("saveAsLibSVMFile") { + val examples = sc.parallelize(Seq( + LabeledPoint(1.1, Vectors.sparse(3, Seq((0, 1.23), (2, 4.56)))), + LabeledPoint(0.0, Vectors.dense(1.01, 2.02, 3.03)) + ), 2) + val tempDir = Files.createTempDir() + val outputDir = new File(tempDir, "output") + MLUtils.saveAsLibSVMFile(examples, outputDir.toURI.toString) + val lines = outputDir.listFiles() + .filter(_.getName.startsWith("part-")) + .flatMap(Source.fromFile(_).getLines()) + .toSet + val expected = Set("1.1 1:1.23 3:4.56", "0.0 1:1.01 2:2.02 3:3.03") + assert(lines === expected) + deleteQuietly(tempDir) + } + + test("appendBias") { + val sv = Vectors.sparse(3, Seq((0, 1.0), (2, 3.0))) + val sv1 = appendBias(sv).asInstanceOf[SparseVector] + assert(sv1.size === 4) + assert(sv1.indices === Array(0, 2, 3)) + assert(sv1.values === Array(1.0, 3.0, 1.0)) + + val dv = Vectors.dense(1.0, 0.0, 3.0) + val dv1 = appendBias(dv).asInstanceOf[DenseVector] + assert(dv1.size === 4) + assert(dv1.values === Array(1.0, 0.0, 3.0, 1.0)) } test("kFold") { val data = sc.parallelize(1 to 100, 2) val collectedData = data.collect().sorted - val twoFoldedRdd = MLUtils.kFold(data, 2, 1) + val twoFoldedRdd = kFold(data, 2, 1) assert(twoFoldedRdd(0)._1.collect().sorted === twoFoldedRdd(1)._2.collect().sorted) assert(twoFoldedRdd(0)._2.collect().sorted === twoFoldedRdd(1)._1.collect().sorted) for (folds <- 2 to 10) { for (seed <- 1 to 5) { - val foldedRdds = MLUtils.kFold(data, folds, seed) + val foldedRdds = kFold(data, folds, seed) assert(foldedRdds.size === folds) foldedRdds.map { case (training, validation) => val result = validation.union(training).collect().sorted @@ -132,4 +158,16 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { } } + /** Delete a file/directory quietly. */ + def deleteQuietly(f: File) { + if (f.isDirectory) { + f.listFiles().foreach(deleteQuietly) + } + try { + f.delete() + } catch { + case _: Throwable => + } + } } + diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index c5844597c95f2..6772e4337ef39 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -90,7 +90,7 @@ class SVMModel(LinearModel): >>> svm.predict(array([1.0])) > 0 True >>> sparse_data = [ - ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), + ... LabeledPoint(0.0, SparseVector(2, {0: -1.0})), ... LabeledPoint(1.0, SparseVector(2, {1: 1.0})), ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) @@ -98,7 +98,7 @@ class SVMModel(LinearModel): >>> svm = SVMWithSGD.train(sc.parallelize(sparse_data)) >>> svm.predict(SparseVector(2, {1: 1.0})) > 0 True - >>> svm.predict(SparseVector(2, {1: 0.0})) <= 0 + >>> svm.predict(SparseVector(2, {0: -1.0})) <= 0 True """ def predict(self, x): From 6d721c5f7131f7c9fe56c524133d70cb37f1222d Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 5 May 2014 19:38:59 -0700 Subject: [PATCH 252/641] [SPARK-1678][SPARK-1679] In-memory compression bug fix and made compression configurable, disabled by default In-memory compression is now configurable in `SparkConf` by the `spark.sql.inMemoryCompression.enabled` property, and is disabled by default. To help code review, the bug fix is in [the first commit](https://github.com/liancheng/spark/commit/d537a367edf0bf24d0b925cc58b21d805ccbc11f), compression configuration is in [the second one](https://github.com/liancheng/spark/commit/4ce09aa8aa820bbbbbaa0f3f084a6cff1d4e6195). Author: Cheng Lian Closes #608 from liancheng/spark-1678 and squashes the following commits: 66c3a8d [Cheng Lian] Renamed in-memory compression configuration key f8fb3a0 [Cheng Lian] Added assertion for testing .hasNext of various decoder 4ce09aa [Cheng Lian] Made in-memory compression configurable via SparkConf d537a36 [Cheng Lian] Fixed SPARK-1678 --- .../org/apache/spark/sql/SQLContext.scala | 7 +++-- .../spark/sql/columnar/ColumnBuilder.scala | 17 ++++++++--- .../columnar/InMemoryColumnarTableScan.scala | 8 +++-- .../sql/columnar/NullableColumnBuilder.scala | 4 +-- .../CompressibleColumnAccessor.scala | 4 ++- .../CompressibleColumnBuilder.scala | 12 +++++++- .../compression/compressionSchemes.scala | 2 +- .../spark/sql/execution/SparkPlan.scala | 2 +- .../scala/org/apache/spark/sql/TestData.scala | 11 +++++++ .../columnar/InMemoryColumnarQuerySuite.scala | 30 +++++++++++++++++-- .../compression/BooleanBitSetSuite.scala | 7 ++++- .../compression/DictionaryEncodingSuite.scala | 7 +++-- .../compression/IntegralDeltaSuite.scala | 7 ++++- .../compression/RunLengthEncodingSuite.scala | 7 +++-- .../TestCompressibleColumnBuilder.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../spark/sql/hive/HiveStrategies.scala | 2 +- 17 files changed, 105 insertions(+), 26 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index e25201a6c1775..bfebfa0c28c52 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -162,8 +162,11 @@ class SQLContext(@transient val sparkContext: SparkContext) /** Caches the specified table in-memory. */ def cacheTable(tableName: String): Unit = { val currentTable = catalog.lookupRelation(None, tableName) + val useCompression = + sparkContext.conf.getBoolean("spark.sql.inMemoryColumnarStorage.compressed", false) val asInMemoryRelation = - InMemoryColumnarTableScan(currentTable.output, executePlan(currentTable).executedPlan) + InMemoryColumnarTableScan( + currentTable.output, executePlan(currentTable).executedPlan, useCompression) catalog.registerTable(None, tableName, SparkLogicalPlan(asInMemoryRelation)) } @@ -173,7 +176,7 @@ class SQLContext(@transient val sparkContext: SparkContext) EliminateAnalysisOperators(catalog.lookupRelation(None, tableName)) match { // This is kind of a hack to make sure that if this was just an RDD registered as a table, // we reregister the RDD as a table. - case SparkLogicalPlan(inMem @ InMemoryColumnarTableScan(_, e: ExistingRdd)) => + case SparkLogicalPlan(inMem @ InMemoryColumnarTableScan(_, e: ExistingRdd, _)) => inMem.cachedColumnBuffers.unpersist() catalog.unregisterTable(None, tableName) catalog.registerTable(None, tableName, SparkLogicalPlan(e)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 048ee66bff44b..4be048cd742d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -28,7 +28,7 @@ private[sql] trait ColumnBuilder { /** * Initializes with an approximate lower bound on the expected number of elements in this column. */ - def initialize(initialSize: Int, columnName: String = "") + def initialize(initialSize: Int, columnName: String = "", useCompression: Boolean = false) /** * Appends `row(ordinal)` to the column builder. @@ -55,7 +55,11 @@ private[sql] class BasicColumnBuilder[T <: DataType, JvmType]( protected var buffer: ByteBuffer = _ - override def initialize(initialSize: Int, columnName: String = "") = { + override def initialize( + initialSize: Int, + columnName: String = "", + useCompression: Boolean = false) = { + val size = if (initialSize == 0) DEFAULT_INITIAL_BUFFER_SIZE else initialSize this.columnName = columnName @@ -130,7 +134,12 @@ private[sql] object ColumnBuilder { } } - def apply(typeId: Int, initialSize: Int = 0, columnName: String = ""): ColumnBuilder = { + def apply( + typeId: Int, + initialSize: Int = 0, + columnName: String = "", + useCompression: Boolean = false): ColumnBuilder = { + val builder = (typeId match { case INT.typeId => new IntColumnBuilder case LONG.typeId => new LongColumnBuilder @@ -144,7 +153,7 @@ private[sql] object ColumnBuilder { case GENERIC.typeId => new GenericColumnBuilder }).asInstanceOf[ColumnBuilder] - builder.initialize(initialSize, columnName) + builder.initialize(initialSize, columnName, useCompression) builder } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index 8a24733047423..fdf28e1bb1261 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -20,8 +20,12 @@ package org.apache.spark.sql.columnar import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute} import org.apache.spark.sql.execution.{SparkPlan, LeafNode} import org.apache.spark.sql.Row +import org.apache.spark.SparkConf -private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], child: SparkPlan) +private[sql] case class InMemoryColumnarTableScan( + attributes: Seq[Attribute], + child: SparkPlan, + useCompression: Boolean) extends LeafNode { override def output: Seq[Attribute] = attributes @@ -30,7 +34,7 @@ private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], ch val output = child.output val cached = child.execute().mapPartitions { iterator => val columnBuilders = output.map { attribute => - ColumnBuilder(ColumnType(attribute.dataType).typeId, 0, attribute.name) + ColumnBuilder(ColumnType(attribute.dataType).typeId, 0, attribute.name, useCompression) }.toArray var row: Row = null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala index 2a3b6fc1e46d3..d008806eedbe1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala @@ -40,12 +40,12 @@ private[sql] trait NullableColumnBuilder extends ColumnBuilder { private var pos: Int = _ private var nullCount: Int = _ - abstract override def initialize(initialSize: Int, columnName: String) { + abstract override def initialize(initialSize: Int, columnName: String, useCompression: Boolean) { nulls = ByteBuffer.allocate(1024) nulls.order(ByteOrder.nativeOrder()) pos = 0 nullCount = 0 - super.initialize(initialSize, columnName) + super.initialize(initialSize, columnName, useCompression) } abstract override def appendFrom(row: Row, ordinal: Int) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala index 878cb84de106f..b4120a3d4368b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala @@ -32,5 +32,7 @@ private[sql] trait CompressibleColumnAccessor[T <: NativeType] extends ColumnAcc decoder = CompressionScheme(underlyingBuffer.getInt()).decoder(buffer, columnType) } - abstract override def extractSingle(buffer: ByteBuffer): T#JvmType = decoder.next() + abstract override def hasNext = super.hasNext || decoder.hasNext + + override def extractSingle(buffer: ByteBuffer): T#JvmType = decoder.next() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index 0f808f68f2eec..4c6675c3c87bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -47,7 +47,17 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] import CompressionScheme._ - val compressionEncoders = schemes.filter(_.supports(columnType)).map(_.encoder[T]) + var compressionEncoders: Seq[Encoder[T]] = _ + + abstract override def initialize(initialSize: Int, columnName: String, useCompression: Boolean) { + compressionEncoders = + if (useCompression) { + schemes.filter(_.supports(columnType)).map(_.encoder[T]) + } else { + Seq(PassThrough.encoder) + } + super.initialize(initialSize, columnName, useCompression) + } protected def isWorthCompressing(encoder: Encoder[T]) = { encoder.compressionRatio < 0.8 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index 800009d3195e1..8cf9ec74ca2de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -157,7 +157,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { currentValue } - override def hasNext = buffer.hasRemaining + override def hasNext = valueCount < run || buffer.hasRemaining } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 50124dd407447..235a9b1692460 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -77,7 +77,7 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan) SparkLogicalPlan( alreadyPlanned match { case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) - case scan @ InMemoryColumnarTableScan(output, child) => + case scan @ InMemoryColumnarTableScan(output, _, _) => scan.copy(attributes = output.map(_.newInstance)) case _ => sys.error("Multiple instance of the same relation detected.") }).asInstanceOf[this.type] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 002b7f0adafab..b5973c0f51be8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -73,4 +73,15 @@ object TestData { ArrayData(Seq(1,2,3), Seq(Seq(1,2,3))) :: ArrayData(Seq(2,3,4), Seq(Seq(2,3,4))) :: Nil) arrayData.registerAsTable("arrayData") + + case class StringData(s: String) + val repeatedData = + TestSQLContext.sparkContext.parallelize(List.fill(2)(StringData("test"))) + repeatedData.registerAsTable("repeatedData") + + val nullableRepeatedData = + TestSQLContext.sparkContext.parallelize( + List.fill(2)(StringData(null)) ++ + List.fill(2)(StringData("test"))) + nullableRepeatedData.registerAsTable("nullableRepeatedData") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 16a13b8a74960..31c5dfba92954 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -28,14 +28,14 @@ class InMemoryColumnarQuerySuite extends QueryTest { test("simple columnar query") { val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan - val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan)) + val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan, true)) checkAnswer(scan, testData.collect().toSeq) } test("projection") { val plan = TestSQLContext.executePlan(testData.select('value, 'key).logicalPlan).executedPlan - val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan)) + val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan, true)) checkAnswer(scan, testData.collect().map { case Row(key: Int, value: String) => value -> key @@ -44,9 +44,33 @@ class InMemoryColumnarQuerySuite extends QueryTest { test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan - val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan)) + val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan, true)) checkAnswer(scan, testData.collect().toSeq) checkAnswer(scan, testData.collect().toSeq) } + + test("SPARK-1678 regression: compression must not lose repeated values") { + checkAnswer( + sql("SELECT * FROM repeatedData"), + repeatedData.collect().toSeq) + + TestSQLContext.cacheTable("repeatedData") + + checkAnswer( + sql("SELECT * FROM repeatedData"), + repeatedData.collect().toSeq) + } + + test("with null values") { + checkAnswer( + sql("SELECT * FROM nullableRepeatedData"), + nullableRepeatedData.collect().toSeq) + + TestSQLContext.cacheTable("nullableRepeatedData") + + checkAnswer( + sql("SELECT * FROM nullableRepeatedData"), + nullableRepeatedData.collect().toSeq) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala index a754f98f7fbf1..93259a19b9fe7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala @@ -72,7 +72,12 @@ class BooleanBitSetSuite extends FunSuite { buffer.rewind().position(headerSize + 4) val decoder = BooleanBitSet.decoder(buffer, BOOLEAN) - values.foreach(expectResult(_, "Wrong decoded value")(decoder.next())) + if (values.nonEmpty) { + values.foreach { + assert(decoder.hasNext) + expectResult(_, "Wrong decoded value")(decoder.next()) + } + } assert(!decoder.hasNext) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala index eab27987e08ea..198dcd8819341 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -98,8 +98,11 @@ class DictionaryEncodingSuite extends FunSuite { val decoder = DictionaryEncoding.decoder(buffer, columnType) - inputSeq.foreach { i => - expectResult(values(i), "Wrong decoded value")(decoder.next()) + if (inputSeq.nonEmpty) { + inputSeq.foreach { i => + assert(decoder.hasNext) + expectResult(values(i), "Wrong decoded value")(decoder.next()) + } } assert(!decoder.hasNext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala index ce419ca7269ba..46af6e001c633 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -96,7 +96,12 @@ class IntegralDeltaSuite extends FunSuite { buffer.rewind().position(headerSize + 4) val decoder = scheme.decoder(buffer, columnType) - input.foreach(expectResult(_, "Wrong decoded value")(decoder.next())) + if (input.nonEmpty) { + input.foreach{ + assert(decoder.hasNext) + expectResult(_, "Wrong decoded value")(decoder.next()) + } + } assert(!decoder.hasNext) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala index 89f9b60a4397b..d3b73ba19d476 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -81,8 +81,11 @@ class RunLengthEncodingSuite extends FunSuite { val decoder = RunLengthEncoding.decoder(buffer, columnType) - inputSeq.foreach { i => - expectResult(values(i), "Wrong decoded value")(decoder.next()) + if (inputSeq.nonEmpty) { + inputSeq.foreach { i => + assert(decoder.hasNext) + expectResult(values(i), "Wrong decoded value")(decoder.next()) + } } assert(!decoder.hasNext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala index 81bf5e99d19b9..6d688ea95cfc0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala @@ -38,7 +38,7 @@ object TestCompressibleColumnBuilder { scheme: CompressionScheme) = { val builder = new TestCompressibleColumnBuilder(columnStats, columnType, Seq(scheme)) - builder.initialize(0) + builder.initialize(0, "", useCompression = true) builder } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 6c907887db79e..ba837a274c51c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -130,7 +130,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with castChildOutput(p, table, child) case p @ logical.InsertIntoTable(SparkLogicalPlan(InMemoryColumnarTableScan( - _, HiveTableScan(_, table, _))), _, child, _) => + _, HiveTableScan(_, table, _), _)), _, child, _) => castChildOutput(p, table, child) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index d9a6e0e88932e..b2157074a41bf 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -45,7 +45,7 @@ private[hive] trait HiveStrategies { case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil case logical.InsertIntoTable(SparkLogicalPlan(InMemoryColumnarTableScan( - _, HiveTableScan(_, table, _))), partition, child, overwrite) => + _, HiveTableScan(_, table, _), _)), partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil case _ => Nil } From a2262cdb7aa30e9f45043f1440d4b02bc3340f9f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 5 May 2014 22:14:47 -0700 Subject: [PATCH 253/641] [SPARK-1735] Add the missing special profiles to make-distribution.sh 73b0cbcc241cca3d318ff74340e80b02f884acbd introduced a few special profiles that are not covered in the `make-distribution.sh`. This affects hadoop versions 2.2.x, 2.3.x, and 2.4.x. Without these special profiles, a java version error for protobufs is thrown at run time. I took the opportunity to rewrite the way we construct the maven command. Previously, the only hadoop version that triggered the `yarn-alpha` profile was 0.23.x, which was inconsistent with the [docs](https://github.com/apache/spark/blob/master/docs/building-with-maven.md). This is now generalized to hadoop versions from 0.23.x to 2.1.x. Author: Andrew Or Closes #660 from andrewor14/hadoop-distribution and squashes the following commits: 6740126 [Andrew Or] Generalize the yarn profile to hadoop versions 2.2+ 88f192d [Andrew Or] Add the required special profiles to make-distribution.sh --- make-distribution.sh | 47 +++++++++++++++++++++++++------------------- 1 file changed, 27 insertions(+), 20 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index ff18d01e7a616..8a63133bc45d4 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -47,7 +47,7 @@ set -o pipefail VERSION=$(mvn help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1) if [ $? != 0 ]; then echo -e "You need Maven installed to build Spark." - echo -e "Download Maven from https://maven.apache.org." + echo -e "Download Maven from https://maven.apache.org/" exit -1; fi @@ -131,27 +131,34 @@ cd $FWDIR export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" -if [ "$SPARK_HIVE" == "true" ]; then - MAYBE_HIVE="-Phive" -else - MAYBE_HIVE="" -fi - -if [ "$SPARK_YARN" == "true" ]; then - if [[ "$SPARK_HADOOP_VERSION" =~ "0.23." ]]; then - mvn clean package -DskipTests -Pyarn-alpha -Dhadoop.version=$SPARK_HADOOP_VERSION \ - -Dyarn.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE -Phadoop-0.23 - else - mvn clean package -DskipTests -Pyarn -Dhadoop.version=$SPARK_HADOOP_VERSION \ - -Dyarn.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE - fi -else - if [[ "$SPARK_HADOOP_VERSION" =~ "0.23." ]]; then - mvn clean package -Phadoop-0.23 -DskipTests -Dhadoop.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE - else - mvn clean package -DskipTests -Dhadoop.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE +BUILD_COMMAND="mvn clean package" + +# Use special profiles for hadoop versions 0.23.x, 2.2.x, 2.3.x, 2.4.x +if [[ "$SPARK_HADOOP_VERSION" =~ ^0\.23\. ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phadoop-0.23"; fi +if [[ "$SPARK_HADOOP_VERSION" =~ ^2\.2\. ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phadoop-2.2"; fi +if [[ "$SPARK_HADOOP_VERSION" =~ ^2\.3\. ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phadoop-2.3"; fi +if [[ "$SPARK_HADOOP_VERSION" =~ ^2\.4\. ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phadoop-2.4"; fi +if [[ "$SPARK_HIVE" == "true" ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phive"; fi +if [[ "$SPARK_YARN" == "true" ]]; then + # For hadoop versions 0.23.x to 2.1.x, use the yarn-alpha profile + if [[ "$SPARK_HADOOP_VERSION" =~ ^0\.2[3-9]\. ]] || + [[ "$SPARK_HADOOP_VERSION" =~ ^0\.[3-9][0-9]\. ]] || + [[ "$SPARK_HADOOP_VERSION" =~ ^1\.[0-9]\. ]] || + [[ "$SPARK_HADOOP_VERSION" =~ ^2\.[0-1]\. ]]; then + BUILD_COMMAND="$BUILD_COMMAND -Pyarn-alpha" + # For hadoop versions 2.2+, use the yarn profile + elif [[ "$SPARK_HADOOP_VERSION" =~ ^2.[2-9]. ]]; then + BUILD_COMMAND="$BUILD_COMMAND -Pyarn" fi + BUILD_COMMAND="$BUILD_COMMAND -Dyarn.version=$SPARK_HADOOP_VERSION" fi +BUILD_COMMAND="$BUILD_COMMAND -Dhadoop.version=$SPARK_HADOOP_VERSION" +BUILD_COMMAND="$BUILD_COMMAND -DskipTests" + +# Actually build the jar +echo -e "\nBuilding with..." +echo -e "\$ $BUILD_COMMAND\n" +${BUILD_COMMAND} # Make directories rm -rf "$DISTDIR" From 3c64750bdd4c2d0a5562f90aead37be81627cc9d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 5 May 2014 22:59:42 -0700 Subject: [PATCH 254/641] [SQL] SPARK-1732 - Support for null primitive values. I also removed a println that I bumped into. Author: Michael Armbrust Closes #658 from marmbrus/nullPrimitives and squashes the following commits: a3ec4f3 [Michael Armbrust] Remove println. 695606b [Michael Armbrust] Support for null primatives from using scala and java reflection. --- .../spark/sql/catalyst/ScalaReflection.scala | 14 ++++- .../spark/sql/api/java/JavaSQLContext.scala | 8 +++ .../org/apache/spark/sql/api/java/Row.scala | 2 +- .../spark/sql/execution/basicOperators.scala | 3 +- .../sql/ScalaReflectionRelationSuite.scala | 34 +++++++++++ .../spark/sql/api/java/JavaSQLSuite.scala | 61 +++++++++++++++++++ .../spark/sql/columnar/ColumnTypeSuite.scala | 5 +- 7 files changed, 122 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 446d0e0bd7f54..792ef6cee6f5d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -44,7 +44,8 @@ object ScalaReflection { case t if t <:< typeOf[Product] => val params = t.member("": TermName).asMethod.paramss StructType( - params.head.map(p => StructField(p.name.toString, schemaFor(p.typeSignature), true))) + params.head.map(p => + StructField(p.name.toString, schemaFor(p.typeSignature), nullable = true))) // Need to decide if we actually need a special type here. case t if t <:< typeOf[Array[Byte]] => BinaryType case t if t <:< typeOf[Array[_]] => @@ -58,6 +59,17 @@ object ScalaReflection { case t if t <:< typeOf[String] => StringType case t if t <:< typeOf[Timestamp] => TimestampType case t if t <:< typeOf[BigDecimal] => DecimalType + case t if t <:< typeOf[Option[_]] => + val TypeRef(_, _, Seq(optType)) = t + schemaFor(optType) + case t if t <:< typeOf[java.lang.Integer] => IntegerType + case t if t <:< typeOf[java.lang.Long] => LongType + case t if t <:< typeOf[java.lang.Double] => DoubleType + case t if t <:< typeOf[java.lang.Float] => FloatType + case t if t <:< typeOf[java.lang.Short] => ShortType + case t if t <:< typeOf[java.lang.Byte] => ByteType + case t if t <:< typeOf[java.lang.Boolean] => BooleanType + // TODO: The following datatypes could be marked as non-nullable. case t if t <:< definitions.IntTpe => IntegerType case t if t <:< definitions.LongTpe => LongType case t if t <:< definitions.DoubleTpe => DoubleType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index a7347088794a8..57facbe10fc96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -132,6 +132,14 @@ class JavaSQLContext(sparkContext: JavaSparkContext) { case c: Class[_] if c == java.lang.Byte.TYPE => ByteType case c: Class[_] if c == java.lang.Float.TYPE => FloatType case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType + + case c: Class[_] if c == classOf[java.lang.Short] => ShortType + case c: Class[_] if c == classOf[java.lang.Integer] => IntegerType + case c: Class[_] if c == classOf[java.lang.Long] => LongType + case c: Class[_] if c == classOf[java.lang.Double] => DoubleType + case c: Class[_] if c == classOf[java.lang.Byte] => ByteType + case c: Class[_] if c == classOf[java.lang.Float] => FloatType + case c: Class[_] if c == classOf[java.lang.Boolean] => BooleanType } // TODO: Nullability could be stricter. AttributeReference(property.getName, dataType, nullable = true)() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala index 362fe769581d7..9b0dd2176149b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{Row => ScalaRow} /** * A result row from a SparkSQL query. */ -class Row(row: ScalaRow) extends Serializable { +class Row(private[spark] val row: ScalaRow) extends Serializable { /** Returns the number of columns present in this Row. */ def length: Int = row.length diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index d807187a5ffb8..8969794c69933 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -164,6 +164,7 @@ case class Sort( @DeveloperApi object ExistingRdd { def convertToCatalyst(a: Any): Any = a match { + case o: Option[_] => o.orNull case s: Seq[Any] => s.map(convertToCatalyst) case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray) case other => other @@ -180,7 +181,7 @@ object ExistingRdd { bufferedIterator.map { r => var i = 0 while (i < mutableRow.length) { - mutableRow(i) = r.productElement(i) + mutableRow(i) = convertToCatalyst(r.productElement(i)) i += 1 } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 1cbf973c34917..f2934da9a031d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -36,6 +36,24 @@ case class ReflectData( timestampField: Timestamp, seqInt: Seq[Int]) +case class NullReflectData( + intField: java.lang.Integer, + longField: java.lang.Long, + floatField: java.lang.Float, + doubleField: java.lang.Double, + shortField: java.lang.Short, + byteField: java.lang.Byte, + booleanField: java.lang.Boolean) + +case class OptionalReflectData( + intField: Option[Int], + longField: Option[Long], + floatField: Option[Float], + doubleField: Option[Double], + shortField: Option[Short], + byteField: Option[Byte], + booleanField: Option[Boolean]) + case class ReflectBinary(data: Array[Byte]) class ScalaReflectionRelationSuite extends FunSuite { @@ -48,6 +66,22 @@ class ScalaReflectionRelationSuite extends FunSuite { assert(sql("SELECT * FROM reflectData").collect().head === data.productIterator.toSeq) } + test("query case class RDD with nulls") { + val data = NullReflectData(null, null, null, null, null, null, null) + val rdd = sparkContext.parallelize(data :: Nil) + rdd.registerAsTable("reflectNullData") + + assert(sql("SELECT * FROM reflectNullData").collect().head === Seq.fill(7)(null)) + } + + test("query case class RDD with Nones") { + val data = OptionalReflectData(None, None, None, None, None, None, None) + val rdd = sparkContext.parallelize(data :: Nil) + rdd.registerAsTable("reflectOptionalData") + + assert(sql("SELECT * FROM reflectOptionalData").collect().head === Seq.fill(7)(null)) + } + // Equality is broken for Arrays, so we test that separately. test("query binary data") { val rdd = sparkContext.parallelize(ReflectBinary(Array[Byte](1)) :: Nil) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala index def0e046a3831..9fff7222fe840 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala @@ -35,6 +35,17 @@ class PersonBean extends Serializable { var age: Int = _ } +class AllTypesBean extends Serializable { + @BeanProperty var stringField: String = _ + @BeanProperty var intField: java.lang.Integer = _ + @BeanProperty var longField: java.lang.Long = _ + @BeanProperty var floatField: java.lang.Float = _ + @BeanProperty var doubleField: java.lang.Double = _ + @BeanProperty var shortField: java.lang.Short = _ + @BeanProperty var byteField: java.lang.Byte = _ + @BeanProperty var booleanField: java.lang.Boolean = _ +} + class JavaSQLSuite extends FunSuite { val javaCtx = new JavaSparkContext(TestSQLContext.sparkContext) val javaSqlCtx = new JavaSQLContext(javaCtx) @@ -50,4 +61,54 @@ class JavaSQLSuite extends FunSuite { schemaRDD.registerAsTable("people") javaSqlCtx.sql("SELECT * FROM people").collect() } + + test("all types in JavaBeans") { + val bean = new AllTypesBean + bean.setStringField("") + bean.setIntField(0) + bean.setLongField(0) + bean.setFloatField(0.0F) + bean.setDoubleField(0.0) + bean.setShortField(0.toShort) + bean.setByteField(0.toByte) + bean.setBooleanField(false) + + val rdd = javaCtx.parallelize(bean :: Nil) + val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[AllTypesBean]) + schemaRDD.registerAsTable("allTypes") + + assert( + javaSqlCtx.sql( + """ + |SELECT stringField, intField, longField, floatField, doubleField, shortField, byteField, + | booleanField + |FROM allTypes + """.stripMargin).collect.head.row === + Seq("", 0, 0L, 0F, 0.0, 0.toShort, 0.toByte, false)) + } + + test("all types null in JavaBeans") { + val bean = new AllTypesBean + bean.setStringField(null) + bean.setIntField(null) + bean.setLongField(null) + bean.setFloatField(null) + bean.setDoubleField(null) + bean.setShortField(null) + bean.setByteField(null) + bean.setBooleanField(null) + + val rdd = javaCtx.parallelize(bean :: Nil) + val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[AllTypesBean]) + schemaRDD.registerAsTable("allTypes") + + assert( + javaSqlCtx.sql( + """ + |SELECT stringField, intField, longField, floatField, doubleField, shortField, byteField, + | booleanField + |FROM allTypes + """.stripMargin).collect.head.row === + Seq.fill(8)(null)) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 325173cf95fdf..71be41056768f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -21,11 +21,12 @@ import java.nio.ByteBuffer import org.scalatest.FunSuite +import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ import org.apache.spark.sql.execution.SparkSqlSerializer -class ColumnTypeSuite extends FunSuite { +class ColumnTypeSuite extends FunSuite with Logging { val DEFAULT_BUFFER_SIZE = 512 test("defaultSize") { @@ -163,7 +164,7 @@ class ColumnTypeSuite extends FunSuite { buffer.rewind() seq.foreach { expected => - println("buffer = " + buffer + ", expected = " + expected) + logger.info("buffer = " + buffer + ", expected = " + expected) val extracted = columnType.extract(buffer) assert( expected === extracted, From 0a5a46811475a7accac1ed8dfefeeba9554456db Mon Sep 17 00:00:00 2001 From: ArcherShao Date: Tue, 6 May 2014 10:12:59 -0700 Subject: [PATCH 255/641] Update OpenHashSet.scala Modify wrong comment of function addWithoutResize. Author: ArcherShao Closes #667 from ArcherShao/patch-3 and squashes the following commits: a607358 [ArcherShao] Update OpenHashSet.scala --- .../scala/org/apache/spark/util/collection/OpenHashSet.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index 19af4f8cbe428..4e363b74f4bef 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -115,10 +115,10 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( * The caller is responsible for calling rehashIfNeeded. * * Use (retval & POSITION_MASK) to get the actual position, and - * (retval & EXISTENCE_MASK) != 0 for prior existence. + * (retval & NONEXISTENCE_MASK) == 0 for prior existence. * * @return The position where the key is placed, plus the highest order bit is set if the key - * exists previously. + * does not exists previously. */ def addWithoutResize(k: T): Int = { var pos = hashcode(hasher.hash(k)) & _mask From 1e829905c791fbf1dfd8e0c1caa62ead7354605e Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 6 May 2014 12:00:09 -0700 Subject: [PATCH 256/641] SPARK-1474: Spark on yarn assembly doesn't include AmIpFilter We use org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter in spark on yarn but are not included it in the assembly jar. I tested this on yarn cluster by removing the yarn jars from the classpath and spark runs fine now. Author: Thomas Graves Closes #406 from tgravescs/SPARK-1474 and squashes the following commits: 1548bf9 [Thomas Graves] SPARK-1474: Spark on yarn assembly doesn't include org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter --- pom.xml | 25 ++++++++++++++++++++++++- project/SparkBuild.scala | 3 ++- yarn/pom.xml | 4 ++++ 3 files changed, 30 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index e4b5c36d69176..fb904e868cfaf 100644 --- a/pom.xml +++ b/pom.xml @@ -616,7 +616,25 @@ - + + org.apache.hadoop + hadoop-yarn-server-web-proxy + ${yarn.version} + + + asm + asm + + + org.ow2.asm + asm + + + org.jboss.netty + netty + + + org.apache.hadoop hadoop-yarn-client @@ -958,6 +976,11 @@ hadoop-yarn-common provided + + org.apache.hadoop + hadoop-yarn-server-web-proxy + provided + org.apache.hadoop hadoop-yarn-client diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a2597e3e6ddd6..1ad05d9e46dd6 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -563,7 +563,8 @@ object SparkBuild extends Build { "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm) + "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-server-web-proxy" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm) ) ) diff --git a/yarn/pom.xml b/yarn/pom.xml index e0094738266b1..6993c89525d8c 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -43,6 +43,10 @@ org.apache.hadoop hadoop-yarn-common + + org.apache.hadoop + hadoop-yarn-server-web-proxy + org.apache.hadoop hadoop-yarn-client From 7b978c1ac59718b85e512c46105b6af641afc3dc Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 6 May 2014 12:07:46 -0700 Subject: [PATCH 257/641] Fix two download suggestions in the docs: 1) On the quick start page provide a direct link to the downloads (suggested by @pbailis). 2) On the index page, don't suggest users always have to build Spark, since many won't. Author: Patrick Wendell Closes #662 from pwendell/quick-start and squashes the following commits: 0622f27 [Patrick Wendell] Fix two download suggestions in the docs: --- docs/index.md | 36 ++++++++++-------------------------- docs/quick-start.md | 8 +++----- 2 files changed, 13 insertions(+), 31 deletions(-) diff --git a/docs/index.md b/docs/index.md index 2daa208b3b903..e3647717a1f18 100644 --- a/docs/index.md +++ b/docs/index.md @@ -9,17 +9,18 @@ It also supports a rich set of higher-level tools including [Shark](http://shark # Downloading -Get Spark by visiting the [downloads page](http://spark.apache.org/downloads.html) of the Apache Spark site. This documentation is for Spark version {{site.SPARK_VERSION}}. +Get Spark by visiting the [downloads page](http://spark.apache.org/downloads.html) of the Apache Spark site. This documentation is for Spark version {{site.SPARK_VERSION}}. The downloads page +contains Spark packages for many popular HDFS versions. If you'd like to build Spark from +scratch, visit the [building with Maven](building-with-maven.html) page. -Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). All you need to run it is to have `java` to installed on your system `PATH`, or the `JAVA_HOME` environment variable pointing to a Java installation. +Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). All you need to run it is +to have `java` to installed on your system `PATH`, or the `JAVA_HOME` environment variable +pointing to a Java installation. -# Building - -Spark uses [Simple Build Tool](http://www.scala-sbt.org), which is bundled with it. To compile the code, go into the top-level Spark directory and run - - sbt/sbt assembly - -For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_BINARY_VERSION}}. If you write applications in Scala, you will need to use a compatible Scala version (e.g. {{site.SCALA_BINARY_VERSION}}.X) -- newer major versions may not work. You can get the right version of Scala from [scala-lang.org](http://www.scala-lang.org/download/). +For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_BINARY_VERSION}}. +If you write applications in Scala, you will need to use a compatible Scala version +(e.g. {{site.SCALA_BINARY_VERSION}}.X) -- newer major versions may not work. You can get the +right version of Scala from [scala-lang.org](http://www.scala-lang.org/download/). # Running the Examples and Shell @@ -50,23 +51,6 @@ options for deployment: * [Apache Mesos](running-on-mesos.html) * [Hadoop YARN](running-on-yarn.html) -# A Note About Hadoop Versions - -Spark uses the Hadoop-client library to talk to HDFS and other Hadoop-supported -storage systems. Because the HDFS protocol has changed in different versions of -Hadoop, you must build Spark against the same version that your cluster uses. -By default, Spark links to Hadoop 1.0.4. You can change this by setting the -`SPARK_HADOOP_VERSION` variable when compiling: - - SPARK_HADOOP_VERSION=2.2.0 sbt/sbt assembly - -In addition, if you wish to run Spark on [YARN](running-on-yarn.html), set -`SPARK_YARN` to `true`: - - SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly - -Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`. - # Where to Go from Here **Programming guides:** diff --git a/docs/quick-start.md b/docs/quick-start.md index 64996b52e0404..478b790f92e17 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -9,11 +9,9 @@ title: Quick Start This tutorial provides a quick introduction to using Spark. We will first introduce the API through Spark's interactive Scala shell (don't worry if you don't know Scala -- you will not need much for this), then show how to write standalone applications in Scala, Java, and Python. See the [programming guide](scala-programming-guide.html) for a more complete reference. -To follow along with this guide, you only need to have successfully built Spark on one machine. Simply go into your Spark directory and run: - -{% highlight bash %} -$ sbt/sbt assembly -{% endhighlight %} +To follow along with this guide, first download a packaged release of Spark from the +[Spark website](http://spark.apache.org/downloads.html). Since we won't be using HDFS, +you can download a package for any version of Hadoop. # Interactive Analysis with the Spark Shell From fbfe69de69aa6767d95167711622ec34c59a1f6d Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 6 May 2014 12:53:39 -0700 Subject: [PATCH 258/641] [SPARK-1685] Cancel retryTimer on restart of Worker or AppClient See https://issues.apache.org/jira/browse/SPARK-1685 for a more complete description, but in essence: If the Worker or AppClient actor restarts before successfully registering with Master, multiple retryTimers will be running, which will lead to less than the full number of registration retries being attempted before the new actor is forced to give up. Author: Mark Hamstra Closes #602 from markhamstra/SPARK-1685 and squashes the following commits: 11cc088 [Mark Hamstra] retryTimer -> registrationRetryTimer 69c348c [Mark Hamstra] Cancel retryTimer on restart of Worker or AppClient --- .../org/apache/spark/deploy/client/AppClient.scala | 13 +++++++++---- .../org/apache/spark/deploy/worker/Worker.scala | 10 ++++++---- 2 files changed, 15 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 57085fc337148..896913d796604 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -60,6 +60,7 @@ private[spark] class AppClient( var master: ActorSelection = null var alreadyDisconnected = false // To avoid calling listener.disconnected() multiple times var alreadyDead = false // To avoid calling listener.dead() multiple times + var registrationRetryTimer: Option[Cancellable] = None override def preStart() { context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) @@ -83,21 +84,20 @@ private[spark] class AppClient( def registerWithMaster() { tryRegisterAllMasters() - import context.dispatcher var retries = 0 - lazy val retryTimer: Cancellable = + registrationRetryTimer = Some { context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { retries += 1 if (registered) { - retryTimer.cancel() + registrationRetryTimer.foreach(_.cancel()) } else if (retries >= REGISTRATION_RETRIES) { markDead("All masters are unresponsive! Giving up.") } else { tryRegisterAllMasters() } } - retryTimer // start timer + } } def changeMaster(url: String) { @@ -177,6 +177,11 @@ private[spark] class AppClient( alreadyDead = true } } + + override def postStop() { + registrationRetryTimer.foreach(_.cancel()) + } + } def start() { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index cd6bd2cd29fc7..85d25dc7dbfa4 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -100,6 +100,8 @@ private[spark] class Worker( val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) val workerSource = new WorkerSource(this) + var registrationRetryTimer: Option[Cancellable] = None + def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed @@ -161,13 +163,12 @@ private[spark] class Worker( def registerWithMaster() { tryRegisterAllMasters() - var retries = 0 - lazy val retryTimer: Cancellable = + registrationRetryTimer = Some { context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { retries += 1 if (registered) { - retryTimer.cancel() + registrationRetryTimer.foreach(_.cancel()) } else if (retries >= REGISTRATION_RETRIES) { logError("All masters are unresponsive! Giving up.") System.exit(1) @@ -175,7 +176,7 @@ private[spark] class Worker( tryRegisterAllMasters() } } - retryTimer // start timer + } } override def receive = { @@ -344,6 +345,7 @@ private[spark] class Worker( } override def postStop() { + registrationRetryTimer.foreach(_.cancel()) executors.values.foreach(_.kill()) drivers.values.foreach(_.kill()) webUi.stop() From ec09acdd4a72333e1c9c2e9d8e12e9c4c07770c8 Mon Sep 17 00:00:00 2001 From: witgo Date: Tue, 6 May 2014 14:17:39 -0700 Subject: [PATCH 259/641] SPARK-1734: spark-submit throws an exception: Exception in thread "main"... ... java.lang.ClassNotFoundException: org.apache.spark.broadcast.TorrentBroadcastFactory Author: witgo Closes #665 from witgo/SPARK-1734 and squashes the following commits: cacf238 [witgo] SPARK-1734: spark-submit throws an exception: Exception in thread "main" java.lang.ClassNotFoundException: org.apache.spark.broadcast.TorrentBroadcastFactory --- .../scala/org/apache/spark/deploy/SparkSubmitArguments.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 45defb9a3f92d..7031cdd9b4ae0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -333,6 +333,6 @@ object SparkSubmitArguments { val message = s"Failed when loading Spark properties file ${file.getName}" throw new SparkException(message, e) } - properties.stringPropertyNames().toSeq.map(k => (k, properties(k))) + properties.stringPropertyNames().toSeq.map(k => (k, properties(k).trim)) } } From 951a5d939863b42da83ac2569d5e9d7ed680e119 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 6 May 2014 15:12:35 -0700 Subject: [PATCH 260/641] [SPARK-1549] Add Python support to spark-submit This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN. This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging. In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit. In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0. Author: Matei Zaharia Closes #664 from mateiz/py-submit and squashes the following commits: 15e9669 [Matei Zaharia] Fix some uses of path.separator property 051278c [Matei Zaharia] Small style fixes 0afe886 [Matei Zaharia] Add license headers 4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests 15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside 47c0655 [Matei Zaharia] More work to make spark-submit work with Python: d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones --- assembly/pom.xml | 13 -- core/pom.xml | 5 + .../scala/org/apache/spark/SparkEnv.scala | 4 +- .../apache/spark/api/python/PythonUtils.scala | 42 ++++ .../api/python/PythonWorkerFactory.scala | 9 +- .../apache/spark/deploy/PythonRunner.scala | 84 ++++++++ .../org/apache/spark/deploy/SparkSubmit.scala | 183 ++++++++++-------- .../spark/deploy/SparkSubmitArguments.scala | 84 +++++--- .../scala/org/apache/spark/util/Utils.scala | 8 +- .../spark/deploy/SparkSubmitSuite.scala | 6 +- docs/python-programming-guide.md | 28 +-- project/SparkBuild.scala | 4 +- python/pyspark/context.py | 6 + python/pyspark/java_gateway.py | 89 ++++----- python/pyspark/tests.py | 131 ++++++++++++- .../org/apache/spark/repl/ReplSuite.scala | 3 +- 16 files changed, 505 insertions(+), 194 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala diff --git a/assembly/pom.xml b/assembly/pom.xml index bdb38806492a6..7d123fb1d7f02 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -40,14 +40,6 @@ root - - - - lib - file://${project.basedir}/lib - - - org.apache.spark @@ -84,11 +76,6 @@ spark-sql_${scala.binary.version} ${project.version} - - net.sf.py4j - py4j - 0.8.1 - diff --git a/core/pom.xml b/core/pom.xml index c24c7be204087..8fe215ab24289 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -247,6 +247,11 @@ pyrolite 2.0.1 + + net.sf.py4j + py4j + 0.8.1 + target/scala-${scala.binary.version}/classes diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index d40ed27da5392..806e77d98fc5f 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -17,6 +17,8 @@ package org.apache.spark +import java.io.File + import scala.collection.JavaConversions._ import scala.collection.mutable import scala.concurrent.Await @@ -304,7 +306,7 @@ object SparkEnv extends Logging { k == "java.class.path" }.getOrElse(("", "")) val classPathEntries = classPathProperty._2 - .split(conf.get("path.separator", ":")) + .split(File.pathSeparator) .filterNot(e => e.isEmpty) .map(e => (e, "System Classpath")) val addedJarsAndFiles = (addedJars ++ addedFiles).map((_, "Added By User")) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala new file mode 100644 index 0000000000000..cf69fa1d53fde --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.python + +import java.io.File + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.SparkContext + +private[spark] object PythonUtils { + /** Get the PYTHONPATH for PySpark, either from SPARK_HOME, if it is set, or from our JAR */ + def sparkPythonPath: String = { + val pythonPath = new ArrayBuffer[String] + for (sparkHome <- sys.env.get("SPARK_HOME")) { + pythonPath += Seq(sparkHome, "python").mkString(File.separator) + pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.8.1-src.zip").mkString(File.separator) + } + pythonPath ++= SparkContext.jarOfObject(this) + pythonPath.mkString(File.pathSeparator) + } + + /** Merge PYTHONPATHS with the appropriate separator. Ignores blank strings. */ + def mergePythonPaths(paths: String*): String = { + paths.filter(_ != "").mkString(File.pathSeparator) + } +} diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 02799ce0091b0..b0bf4e052b3e9 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -37,6 +37,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1)) var daemonPort: Int = 0 + val pythonPath = PythonUtils.mergePythonPaths( + PythonUtils.sparkPythonPath, envVars.getOrElse("PYTHONPATH", "")) + def create(): Socket = { if (useDaemon) { createThroughDaemon() @@ -78,9 +81,10 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String serverSocket = new ServerSocket(0, 1, InetAddress.getByAddress(Array(127, 0, 0, 1))) // Create and start the worker - val pb = new ProcessBuilder(Seq(pythonExec, "-m", "pyspark.worker")) + val pb = new ProcessBuilder(Seq(pythonExec, "-u", "-m", "pyspark.worker")) val workerEnv = pb.environment() workerEnv.putAll(envVars) + workerEnv.put("PYTHONPATH", pythonPath) val worker = pb.start() // Redirect the worker's stderr to ours @@ -151,9 +155,10 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String try { // Create and start the daemon - val pb = new ProcessBuilder(Seq(pythonExec, "-m", "pyspark.daemon")) + val pb = new ProcessBuilder(Seq(pythonExec, "-u", "-m", "pyspark.daemon")) val workerEnv = pb.environment() workerEnv.putAll(envVars) + workerEnv.put("PYTHONPATH", pythonPath) daemon = pb.start() // Redirect the stderr to ours diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala new file mode 100644 index 0000000000000..f2e7c7a508b3f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -0,0 +1,84 @@ +/* + * 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.deploy + +import java.io.{IOException, File, InputStream, OutputStream} + +import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConversions._ + +import org.apache.spark.SparkContext +import org.apache.spark.api.python.PythonUtils + +/** + * A main class used by spark-submit to launch Python applications. It executes python as a + * subprocess and then has it connect back to the JVM to access system properties, etc. + */ +object PythonRunner { + def main(args: Array[String]) { + val primaryResource = args(0) + val pyFiles = args(1) + val otherArgs = args.slice(2, args.length) + + val pythonExec = sys.env.get("PYSPARK_PYTHON").getOrElse("python") // TODO: get this from conf + + // Launch a Py4J gateway server for the process to connect to; this will let it see our + // Java system properties and such + val gatewayServer = new py4j.GatewayServer(null, 0) + gatewayServer.start() + + // Build up a PYTHONPATH that includes the Spark assembly JAR (where this class is), the + // python directories in SPARK_HOME (if set), and any files in the pyFiles argument + val pathElements = new ArrayBuffer[String] + pathElements ++= pyFiles.split(",") + pathElements += PythonUtils.sparkPythonPath + pathElements += sys.env.getOrElse("PYTHONPATH", "") + val pythonPath = PythonUtils.mergePythonPaths(pathElements: _*) + + // Launch Python process + val builder = new ProcessBuilder(Seq(pythonExec, "-u", primaryResource) ++ otherArgs) + val env = builder.environment() + env.put("PYTHONPATH", pythonPath) + env.put("PYSPARK_GATEWAY_PORT", "" + gatewayServer.getListeningPort) + builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize + val process = builder.start() + + new RedirectThread(process.getInputStream, System.out, "redirect output").start() + + System.exit(process.waitFor()) + } + + /** + * A utility class to redirect the child process's stdout or stderr + */ + class RedirectThread(in: InputStream, out: OutputStream, name: String) extends Thread(name) { + setDaemon(true) + override def run() { + scala.util.control.Exception.ignoring(classOf[IOException]) { + // FIXME: We copy the stream on the level of bytes to avoid encoding problems. + val buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + out.write(buf, 0, len) + out.flush() + len = in.read(buf) + } + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index fb30e8a70f682..e39723f38347c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -60,11 +60,11 @@ object SparkSubmit { private[spark] var exitFn: () => Unit = () => System.exit(-1) private[spark] def printErrorAndExit(str: String) = { - printStream.println("error: " + str) - printStream.println("run with --help for more information or --verbose for debugging output") + printStream.println("Error: " + str) + printStream.println("Run with --help for usage help or --verbose for debug output") exitFn() } - private[spark] def printWarning(str: String) = printStream.println("warning: " + str) + private[spark] def printWarning(str: String) = printStream.println("Warning: " + str) /** * @return @@ -72,43 +72,43 @@ object SparkSubmit { * entries for the child, a list of system propertes, a list of env vars * and the main class for the child */ - private[spark] def createLaunchEnv(appArgs: SparkSubmitArguments): (ArrayBuffer[String], + private[spark] def createLaunchEnv(args: SparkSubmitArguments): (ArrayBuffer[String], ArrayBuffer[String], Map[String, String], String) = { - if (appArgs.master.startsWith("local")) { + if (args.master.startsWith("local")) { clusterManager = LOCAL - } else if (appArgs.master.startsWith("yarn")) { + } else if (args.master.startsWith("yarn")) { clusterManager = YARN - } else if (appArgs.master.startsWith("spark")) { + } else if (args.master.startsWith("spark")) { clusterManager = STANDALONE - } else if (appArgs.master.startsWith("mesos")) { + } else if (args.master.startsWith("mesos")) { clusterManager = MESOS } else { - printErrorAndExit("master must start with yarn, mesos, spark, or local") + printErrorAndExit("Master must start with yarn, mesos, spark, or local") } // Because "yarn-cluster" and "yarn-client" encapsulate both the master // and deploy mode, we have some logic to infer the master and deploy mode // from each other if only one is specified, or exit early if they are at odds. - if (appArgs.deployMode == null && - (appArgs.master == "yarn-standalone" || appArgs.master == "yarn-cluster")) { - appArgs.deployMode = "cluster" + if (args.deployMode == null && + (args.master == "yarn-standalone" || args.master == "yarn-cluster")) { + args.deployMode = "cluster" } - if (appArgs.deployMode == "cluster" && appArgs.master == "yarn-client") { + if (args.deployMode == "cluster" && args.master == "yarn-client") { printErrorAndExit("Deploy mode \"cluster\" and master \"yarn-client\" are not compatible") } - if (appArgs.deployMode == "client" && - (appArgs.master == "yarn-standalone" || appArgs.master == "yarn-cluster")) { - printErrorAndExit("Deploy mode \"client\" and master \"" + appArgs.master + if (args.deployMode == "client" && + (args.master == "yarn-standalone" || args.master == "yarn-cluster")) { + printErrorAndExit("Deploy mode \"client\" and master \"" + args.master + "\" are not compatible") } - if (appArgs.deployMode == "cluster" && appArgs.master.startsWith("yarn")) { - appArgs.master = "yarn-cluster" + if (args.deployMode == "cluster" && args.master.startsWith("yarn")) { + args.master = "yarn-cluster" } - if (appArgs.deployMode != "cluster" && appArgs.master.startsWith("yarn")) { - appArgs.master = "yarn-client" + if (args.deployMode != "cluster" && args.master.startsWith("yarn")) { + args.master = "yarn-client" } - val deployOnCluster = Option(appArgs.deployMode).getOrElse("client") == "cluster" + val deployOnCluster = Option(args.deployMode).getOrElse("client") == "cluster" val childClasspath = new ArrayBuffer[String]() val childArgs = new ArrayBuffer[String]() @@ -116,76 +116,93 @@ object SparkSubmit { var childMainClass = "" if (clusterManager == MESOS && deployOnCluster) { - printErrorAndExit("Mesos does not support running the driver on the cluster") + printErrorAndExit("Cannot currently run driver on the cluster in Mesos") } + // If we're running a Python app, set the Java class to run to be our PythonRunner, add + // Python files to deployment list, and pass the main file and Python path to PythonRunner + if (args.isPython) { + if (deployOnCluster) { + printErrorAndExit("Cannot currently run Python driver programs on cluster") + } + args.mainClass = "org.apache.spark.deploy.PythonRunner" + args.files = mergeFileLists(args.files, args.pyFiles, args.primaryResource) + val pyFiles = Option(args.pyFiles).getOrElse("") + args.childArgs = ArrayBuffer(args.primaryResource, pyFiles) ++ args.childArgs + args.primaryResource = RESERVED_JAR_NAME + sysProps("spark.submit.pyFiles") = pyFiles + } + + // If we're deploying into YARN, use yarn.Client as a wrapper around the user class if (!deployOnCluster) { - childMainClass = appArgs.mainClass - if (appArgs.primaryResource != RESERVED_JAR_NAME) { - childClasspath += appArgs.primaryResource + childMainClass = args.mainClass + if (args.primaryResource != RESERVED_JAR_NAME) { + childClasspath += args.primaryResource } } else if (clusterManager == YARN) { childMainClass = "org.apache.spark.deploy.yarn.Client" - childArgs += ("--jar", appArgs.primaryResource) - childArgs += ("--class", appArgs.mainClass) + childArgs += ("--jar", args.primaryResource) + childArgs += ("--class", args.mainClass) } + // Make sure YARN is included in our build if we're trying to use it if (clusterManager == YARN) { - // The choice of class is arbitrary, could use any spark-yarn class if (!Utils.classIsLoadable("org.apache.spark.deploy.yarn.Client") && !Utils.isTesting) { - val msg = "Could not load YARN classes. This copy of Spark may not have been compiled " + - "with YARN support." - throw new Exception(msg) + printErrorAndExit("Could not load YARN classes. " + + "This copy of Spark may not have been compiled with YARN support.") } } // Special flag to avoid deprecation warnings at the client sysProps("SPARK_SUBMIT") = "true" + // A list of rules to map each argument to system properties or command-line options in + // each deploy mode; we iterate through these below val options = List[OptionAssigner]( - new OptionAssigner(appArgs.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), - new OptionAssigner(appArgs.driverExtraClassPath, STANDALONE | YARN, true, + OptionAssigner(args.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), + OptionAssigner(args.driverExtraClassPath, STANDALONE | YARN, true, sysProp = "spark.driver.extraClassPath"), - new OptionAssigner(appArgs.driverExtraJavaOptions, STANDALONE | YARN, true, + OptionAssigner(args.driverExtraJavaOptions, STANDALONE | YARN, true, sysProp = "spark.driver.extraJavaOptions"), - new OptionAssigner(appArgs.driverExtraLibraryPath, STANDALONE | YARN, true, + OptionAssigner(args.driverExtraLibraryPath, STANDALONE | YARN, true, sysProp = "spark.driver.extraLibraryPath"), - new OptionAssigner(appArgs.driverMemory, YARN, true, clOption = "--driver-memory"), - new OptionAssigner(appArgs.name, YARN, true, clOption = "--name"), - new OptionAssigner(appArgs.queue, YARN, true, clOption = "--queue"), - new OptionAssigner(appArgs.queue, YARN, false, sysProp = "spark.yarn.queue"), - new OptionAssigner(appArgs.numExecutors, YARN, true, clOption = "--num-executors"), - new OptionAssigner(appArgs.numExecutors, YARN, false, sysProp = "spark.executor.instances"), - new OptionAssigner(appArgs.executorMemory, YARN, true, clOption = "--executor-memory"), - new OptionAssigner(appArgs.executorMemory, STANDALONE | MESOS | YARN, false, + OptionAssigner(args.driverMemory, YARN, true, clOption = "--driver-memory"), + OptionAssigner(args.name, YARN, true, clOption = "--name"), + OptionAssigner(args.queue, YARN, true, clOption = "--queue"), + OptionAssigner(args.queue, YARN, false, sysProp = "spark.yarn.queue"), + OptionAssigner(args.numExecutors, YARN, true, clOption = "--num-executors"), + OptionAssigner(args.numExecutors, YARN, false, sysProp = "spark.executor.instances"), + OptionAssigner(args.executorMemory, YARN, true, clOption = "--executor-memory"), + OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, false, sysProp = "spark.executor.memory"), - new OptionAssigner(appArgs.driverMemory, STANDALONE, true, clOption = "--memory"), - new OptionAssigner(appArgs.driverCores, STANDALONE, true, clOption = "--cores"), - new OptionAssigner(appArgs.executorCores, YARN, true, clOption = "--executor-cores"), - new OptionAssigner(appArgs.executorCores, YARN, false, sysProp = "spark.executor.cores"), - new OptionAssigner(appArgs.totalExecutorCores, STANDALONE | MESOS, false, + OptionAssigner(args.driverMemory, STANDALONE, true, clOption = "--memory"), + OptionAssigner(args.driverCores, STANDALONE, true, clOption = "--cores"), + OptionAssigner(args.executorCores, YARN, true, clOption = "--executor-cores"), + OptionAssigner(args.executorCores, YARN, false, sysProp = "spark.executor.cores"), + OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, false, sysProp = "spark.cores.max"), - new OptionAssigner(appArgs.files, YARN, false, sysProp = "spark.yarn.dist.files"), - new OptionAssigner(appArgs.files, YARN, true, clOption = "--files"), - new OptionAssigner(appArgs.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), - new OptionAssigner(appArgs.archives, YARN, true, clOption = "--archives"), - new OptionAssigner(appArgs.jars, YARN, true, clOption = "--addJars"), - new OptionAssigner(appArgs.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), - new OptionAssigner(appArgs.jars, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.jars"), - new OptionAssigner(appArgs.name, LOCAL | STANDALONE | MESOS, false, - sysProp = "spark.app.name") + OptionAssigner(args.files, YARN, false, sysProp = "spark.yarn.dist.files"), + OptionAssigner(args.files, YARN, true, clOption = "--files"), + OptionAssigner(args.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), + OptionAssigner(args.archives, YARN, true, clOption = "--archives"), + OptionAssigner(args.jars, YARN, true, clOption = "--addJars"), + OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.files"), + OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), + OptionAssigner(args.jars, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.jars"), + OptionAssigner(args.name, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.app.name") ) // For client mode make any added jars immediately visible on the classpath - if (appArgs.jars != null && !deployOnCluster) { - for (jar <- appArgs.jars.split(",")) { + if (args.jars != null && !deployOnCluster) { + for (jar <- args.jars.split(",")) { childClasspath += jar } } + // Map all arguments to command-line options or system properties for our chosen mode for (opt <- options) { if (opt.value != null && deployOnCluster == opt.deployOnCluster && - (clusterManager & opt.clusterManager) != 0) { + (clusterManager & opt.clusterManager) != 0) { if (opt.clOption != null) { childArgs += (opt.clOption, opt.value) } else if (opt.sysProp != null) { @@ -197,32 +214,35 @@ object SparkSubmit { // For standalone mode, add the application jar automatically so the user doesn't have to // call sc.addJar. TODO: Standalone mode in the cluster if (clusterManager == STANDALONE) { - val existingJars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq()) - sysProps.put("spark.jars", (existingJars ++ Seq(appArgs.primaryResource)).mkString(",")) + var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq()) + if (args.primaryResource != RESERVED_JAR_NAME) { + jars = jars ++ Seq(args.primaryResource) + } + sysProps.put("spark.jars", jars.mkString(",")) } if (deployOnCluster && clusterManager == STANDALONE) { - if (appArgs.supervise) { + if (args.supervise) { childArgs += "--supervise" } childMainClass = "org.apache.spark.deploy.Client" childArgs += "launch" - childArgs += (appArgs.master, appArgs.primaryResource, appArgs.mainClass) + childArgs += (args.master, args.primaryResource, args.mainClass) } // Arguments to be passed to user program - if (appArgs.childArgs != null) { + if (args.childArgs != null) { if (!deployOnCluster || clusterManager == STANDALONE) { - childArgs ++= appArgs.childArgs + childArgs ++= args.childArgs } else if (clusterManager == YARN) { - for (arg <- appArgs.childArgs) { + for (arg <- args.childArgs) { childArgs += ("--arg", arg) } } } - for ((k, v) <- appArgs.getDefaultSparkProperties) { + for ((k, v) <- args.getDefaultSparkProperties) { if (!sysProps.contains(k)) sysProps(k) = v } @@ -230,8 +250,8 @@ object SparkSubmit { } private def launch(childArgs: ArrayBuffer[String], childClasspath: ArrayBuffer[String], - sysProps: Map[String, String], childMainClass: String, verbose: Boolean = false) { - + sysProps: Map[String, String], childMainClass: String, verbose: Boolean = false) + { if (verbose) { printStream.println(s"Main class:\n$childMainClass") printStream.println(s"Arguments:\n${childArgs.mkString("\n")}") @@ -273,15 +293,26 @@ object SparkSubmit { val url = localJarFile.getAbsoluteFile.toURI.toURL loader.addURL(url) } + + /** + * Merge a sequence of comma-separated file lists, some of which may be null to indicate + * no files, into a single comma-separated string. + */ + private[spark] def mergeFileLists(lists: String*): String = { + val merged = lists.filter(_ != null) + .flatMap(_.split(",")) + .mkString(",") + if (merged == "") null else merged + } } /** * Provides an indirection layer for passing arguments as system properties or flags to * the user's driver program or to downstream launcher tools. */ -private[spark] class OptionAssigner(val value: String, - val clusterManager: Int, - val deployOnCluster: Boolean, - val clOption: String = null, - val sysProp: String = null -) { } +private[spark] case class OptionAssigner( + value: String, + clusterManager: Int, + deployOnCluster: Boolean, + clOption: String = null, + sysProp: String = null) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 7031cdd9b4ae0..2d327aa3fb27f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy import java.io.{File, FileInputStream, IOException} import java.util.Properties +import java.util.jar.JarFile import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -52,6 +53,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { var childArgs: ArrayBuffer[String] = new ArrayBuffer[String]() var jars: String = null var verbose: Boolean = false + var isPython: Boolean = false + var pyFiles: String = null parseOpts(args.toList) loadDefaults() @@ -76,7 +79,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { } /** Fill in any undefined values based on the current properties file or built-in defaults. */ - private def loadDefaults() = { + private def loadDefaults(): Unit = { // Use common defaults file, if not specified by user if (propertiesFile == null) { @@ -107,15 +110,43 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { master = Option(master).getOrElse(System.getenv("MASTER")) deployMode = Option(deployMode).getOrElse(System.getenv("DEPLOY_MODE")) + // Try to set main class from JAR if no --class argument is given + if (mainClass == null && !isPython && primaryResource != null) { + try { + val jar = new JarFile(primaryResource) + // Note that this might still return null if no main-class is set; we catch that later + mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class") + } catch { + case e: Exception => + SparkSubmit.printErrorAndExit("Failed to read JAR: " + primaryResource) + return + } + } + // Global defaults. These should be keep to minimum to avoid confusing behavior. master = Option(master).getOrElse("local[*]") + + // Set name from main class if not given + name = Option(name).orElse(Option(mainClass)).orNull + if (name == null && primaryResource != null) { + name = Utils.stripDirectory(primaryResource) + } } /** Ensure that required fields exists. Call this only once all defaults are loaded. */ private def checkRequiredArguments() = { - if (args.length == 0) printUsageAndExit(-1) - if (primaryResource == null) SparkSubmit.printErrorAndExit("Must specify a primary resource") - if (mainClass == null) SparkSubmit.printErrorAndExit("Must specify a main class with --class") + if (args.length == 0) { + printUsageAndExit(-1) + } + if (primaryResource == null) { + SparkSubmit.printErrorAndExit("Must specify a primary resource (JAR or Python file)") + } + if (mainClass == null && !isPython) { + SparkSubmit.printErrorAndExit("No main class set in JAR; please specify one with --class") + } + if (pyFiles != null && !isPython) { + SparkSubmit.printErrorAndExit("--py-files given but primary resource is not a Python script") + } if (master.startsWith("yarn")) { val hasHadoopEnv = sys.env.contains("HADOOP_CONF_DIR") || sys.env.contains("YARN_CONF_DIR") @@ -143,6 +174,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | queue $queue | numExecutors $numExecutors | files $files + | pyFiles $pyFiles | archives $archives | mainClass $mainClass | primaryResource $primaryResource @@ -234,6 +266,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { files = value parse(tail) + case ("--py-files") :: value :: tail => + pyFiles = value + parse(tail) + case ("--archives") :: value :: tail => archives = value parse(tail) @@ -260,9 +296,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { val errMessage = s"Unrecognized option '$value'." SparkSubmit.printErrorAndExit(errMessage) case v => - primaryResource = v - inSparkOpts = false - parse(tail) + primaryResource = v + inSparkOpts = false + isPython = v.endsWith(".py") + parse(tail) } } else { childArgs += value @@ -270,7 +307,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { } case Nil => - } + } } private def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { @@ -279,23 +316,26 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { outStream.println("Unknown/unsupported param " + unknownParam) } outStream.println( - """Usage: spark-submit [options] [app options] + """Usage: spark-submit [options] [app options] |Options: | --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. - | --deploy-mode DEPLOY_MODE Mode to deploy the app in, either 'client' or 'cluster'. - | --class CLASS_NAME Name of your app's main class (required for Java apps). - | --name NAME The name of your application (Default: 'Spark'). - | --jars JARS A comma-separated list of local jars to include on the - | driver classpath and that SparkContext.addJar will work - | with. Doesn't work on standalone with 'cluster' deploy mode. - | --files FILES Comma separated list of files to be placed in the working dir - | of each executor. + | --deploy-mode DEPLOY_MODE Where to run the driver program: either "client" to run + | on the local machine, or "cluster" to run inside cluster. + | --class CLASS_NAME Your application's main class (for Java / Scala apps). + | --name NAME A name of your application. + | --jars JARS Comma-separated list of local jars to include on the driver + | and executor classpaths. Doesn't work for drivers in + | standalone mode with "cluster" deploy mode. + | --py-files PY_FILES Comma-separated list of .zip or .egg files to place on the + | PYTHONPATH for Python apps. + | --files FILES Comma-separated list of files to be placed in the working + | directory of each executor. | --properties-file FILE Path to a file from which to load extra properties. If not | specified, this will look for conf/spark-defaults.conf. | | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). - | --driver-java-options Extra Java options to pass to the driver - | --driver-library-path Extra library path entries to pass to the driver + | --driver-java-options Extra Java options to pass to the driver. + | --driver-library-path Extra library path entries to pass to the driver. | --driver-class-path Extra class path entries to pass to the driver. Note that | jars added with --jars are automatically included in the | classpath. @@ -311,10 +351,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | | YARN-only: | --executor-cores NUM Number of cores per executor (Default: 1). - | --queue QUEUE_NAME The YARN queue to submit to (Default: 'default'). - | --num-executors NUM Number of executors to (Default: 2). + | --queue QUEUE_NAME The YARN queue to submit to (Default: "default"). + | --num-executors NUM Number of executors to launch (Default: 2). | --archives ARCHIVES Comma separated list of archives to be extracted into the - | working dir of each executor.""".stripMargin + | working directory of each executor.""".stripMargin ) SparkSubmit.exitFn() } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index bef4dab3d7cc1..202bd46956f87 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -46,7 +46,6 @@ import org.apache.spark.serializer.{DeserializationStream, SerializationStream, * Various utility methods used by Spark. */ private[spark] object Utils extends Logging { - val random = new Random() def sparkBin(sparkHome: String, which: String): File = { @@ -1082,4 +1081,11 @@ private[spark] object Utils extends Logging { def isTesting = { sys.env.contains("SPARK_TESTING") || sys.props.contains("spark.testing") } + + /** + * Strip the directory from a path name + */ + def stripDirectory(path: String): String = { + path.split(File.separator).last + } } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index b3541b4a40b79..d7e3b22ed476e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -83,7 +83,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handle binary specified but not class") { - testPrematureExit(Array("foo.jar"), "Must specify a main class") + testPrematureExit(Array("foo.jar"), "No main class") } test("handles arguments with --key=val") { @@ -94,9 +94,9 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles arguments to user program") { - val clArgs = Seq("--name", "myApp", "userjar.jar", "some", "--random", "args", "here") + val clArgs = Seq("--name", "myApp", "--class", "Foo", "userjar.jar", "some", "--weird", "args") val appArgs = new SparkSubmitArguments(clArgs) - appArgs.childArgs should be (Seq("some", "--random", "args", "here")) + appArgs.childArgs should be (Seq("some", "--weird", "args")) } test("handles YARN cluster mode") { diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 98c456228af9f..8ea22e15a4b69 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -60,12 +60,9 @@ By default, PySpark requires `python` to be available on the system `PATH` and u All of PySpark's library dependencies, including [Py4J](http://py4j.sourceforge.net/), are bundled with PySpark and automatically imported. -Standalone PySpark applications should be run using the `bin/pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh` or `.cmd`. -The script automatically adds the `bin/pyspark` package to the `PYTHONPATH`. +Standalone PySpark applications should be run using the `bin/spark-submit` script, which automatically +configures the Java and Python environment for running Spark. -# Running PySpark on YARN - -To run PySpark against a YARN cluster, simply set the MASTER environment variable to "yarn-client". # Interactive Use @@ -103,7 +100,7 @@ $ MASTER=local[4] ./bin/pyspark ## IPython -It is also possible to launch PySpark in [IPython](http://ipython.org), the +It is also possible to launch the PySpark shell in [IPython](http://ipython.org), the enhanced Python interpreter. PySpark works with IPython 1.0.0 and later. To use IPython, set the `IPYTHON` variable to `1` when running `bin/pyspark`: @@ -123,18 +120,17 @@ IPython also works on a cluster or on multiple cores if you set the `MASTER` env # Standalone Programs -PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `bin/pyspark`. +PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `bin/spark-submit`. The Quick Start guide includes a [complete example](quick-start.html#a-standalone-app-in-python) of a standalone Python application. -Code dependencies can be deployed by listing them in the `pyFiles` option in the SparkContext constructor: +Code dependencies can be deployed by passing .zip or .egg files in the `--py-files` option of `spark-submit`: -{% highlight python %} -from pyspark import SparkContext -sc = SparkContext("local", "App Name", pyFiles=['MyFile.py', 'lib.zip', 'app.egg']) +{% highlight bash %} +./bin/spark-submit --py-files lib1.zip,lib2.zip my_script.py {% endhighlight %} Files listed here will be added to the `PYTHONPATH` and shipped to remote worker machines. -Code dependencies can be added to an existing SparkContext using its `addPyFile()` method. +Code dependencies can also be added to an existing SparkContext at runtime using its `addPyFile()` method. You can set [configuration properties](configuration.html#spark-properties) by passing a [SparkConf](api/python/pyspark.conf.SparkConf-class.html) object to SparkContext: @@ -142,12 +138,16 @@ You can set [configuration properties](configuration.html#spark-properties) by p {% highlight python %} from pyspark import SparkConf, SparkContext conf = (SparkConf() - .setMaster("local") .setAppName("My app") .set("spark.executor.memory", "1g")) sc = SparkContext(conf = conf) {% endhighlight %} +`spark-submit` supports launching Python applications on standalone, Mesos or YARN clusters, through +its `--master` argument. However, it currently requires the Python driver program to run on the local +machine, not the cluster (i.e. the `--deploy-mode` parameter cannot be `cluster`). + + # API Docs [API documentation](api/python/index.html) for PySpark is available as Epydoc. @@ -164,6 +164,6 @@ some example applications. PySpark also includes several sample programs in the [`python/examples` folder](https://github.com/apache/spark/tree/master/python/examples). You can run them by passing the files to `pyspark`; e.g.: - ./bin/pyspark python/examples/wordcount.py + ./bin/spark-submit python/examples/wordcount.py Each program prints usage help when run without arguments. diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 1ad05d9e46dd6..7f9746ec4acc0 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -356,7 +356,8 @@ object SparkBuild extends Build { "com.twitter" % "chill-java" % chillVersion excludeAll(excludeAsm), "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), "com.clearspring.analytics" % "stream" % "2.5.1" excludeAll(excludeFastutil), - "org.spark-project" % "pyrolite" % "2.0.1" + "org.spark-project" % "pyrolite" % "2.0.1", + "net.sf.py4j" % "py4j" % "0.8.1" ), libraryDependencies ++= maybeAvro ) @@ -569,7 +570,6 @@ object SparkBuild extends Build { ) def assemblyProjSettings = sharedSettings ++ Seq( - libraryDependencies += "net.sf.py4j" % "py4j" % "0.8.1", name := "spark-assembly", assembleDeps in Compile <<= (packageProjects.map(packageBin in Compile in _) ++ Seq(packageDependency in Compile)).dependOn, jarName in assembly <<= version map { v => "spark-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" }, diff --git a/python/pyspark/context.py b/python/pyspark/context.py index c74dc5fd4f854..c7dc85ea03544 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -158,6 +158,12 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, for path in (pyFiles or []): self.addPyFile(path) + # Deploy code dependencies set by spark-submit; these will already have been added + # with SparkContext.addFile, so we just need to add them + for path in self._conf.get("spark.submit.pyFiles", "").split(","): + if path != "": + self._python_includes.append(os.path.basename(path)) + # Create a temporary directory inside spark.local.dir: local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir(self._jsc.sc().conf()) self._temp_dir = \ diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 032d960e40998..3d0936fdca911 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -27,39 +27,43 @@ def launch_gateway(): SPARK_HOME = os.environ["SPARK_HOME"] - set_env_vars_for_yarn() - - # Launch the Py4j gateway using Spark's run command so that we pick up the - # proper classpath and settings from spark-env.sh - on_windows = platform.system() == "Windows" - script = "./bin/spark-class.cmd" if on_windows else "./bin/spark-class" - command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer", - "--die-on-broken-pipe", "0"] - if not on_windows: - # Don't send ctrl-c / SIGINT to the Java gateway: - def preexec_func(): - signal.signal(signal.SIGINT, signal.SIG_IGN) - proc = Popen(command, stdout=PIPE, stdin=PIPE, preexec_fn=preexec_func) + gateway_port = -1 + if "PYSPARK_GATEWAY_PORT" in os.environ: + gateway_port = int(os.environ["PYSPARK_GATEWAY_PORT"]) else: - # preexec_fn not supported on Windows - proc = Popen(command, stdout=PIPE, stdin=PIPE) - # Determine which ephemeral port the server started on: - port = int(proc.stdout.readline()) - # Create a thread to echo output from the GatewayServer, which is required - # for Java log output to show up: - class EchoOutputThread(Thread): - def __init__(self, stream): - Thread.__init__(self) - self.daemon = True - self.stream = stream + # Launch the Py4j gateway using Spark's run command so that we pick up the + # proper classpath and settings from spark-env.sh + on_windows = platform.system() == "Windows" + script = "./bin/spark-class.cmd" if on_windows else "./bin/spark-class" + command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer", + "--die-on-broken-pipe", "0"] + if not on_windows: + # Don't send ctrl-c / SIGINT to the Java gateway: + def preexec_func(): + signal.signal(signal.SIGINT, signal.SIG_IGN) + proc = Popen(command, stdout=PIPE, stdin=PIPE, preexec_fn=preexec_func) + else: + # preexec_fn not supported on Windows + proc = Popen(command, stdout=PIPE, stdin=PIPE) + # Determine which ephemeral port the server started on: + gateway_port = int(proc.stdout.readline()) + # Create a thread to echo output from the GatewayServer, which is required + # for Java log output to show up: + class EchoOutputThread(Thread): + def __init__(self, stream): + Thread.__init__(self) + self.daemon = True + self.stream = stream + + def run(self): + while True: + line = self.stream.readline() + sys.stderr.write(line) + EchoOutputThread(proc.stdout).start() - def run(self): - while True: - line = self.stream.readline() - sys.stderr.write(line) - EchoOutputThread(proc.stdout).start() # Connect to the gateway - gateway = JavaGateway(GatewayClient(port=port), auto_convert=False) + gateway = JavaGateway(GatewayClient(port=gateway_port), auto_convert=False) + # Import the classes used by PySpark java_import(gateway.jvm, "org.apache.spark.SparkConf") java_import(gateway.jvm, "org.apache.spark.api.java.*") @@ -70,28 +74,5 @@ def run(self): java_import(gateway.jvm, "org.apache.spark.sql.hive.LocalHiveContext") java_import(gateway.jvm, "org.apache.spark.sql.hive.TestHiveContext") java_import(gateway.jvm, "scala.Tuple2") - return gateway -def set_env_vars_for_yarn(): - # Add the spark jar, which includes the pyspark files, to the python path - env_map = parse_env(os.environ.get("SPARK_YARN_USER_ENV", "")) - if "PYTHONPATH" in env_map: - env_map["PYTHONPATH"] += ":spark.jar" - else: - env_map["PYTHONPATH"] = "spark.jar" - - os.environ["SPARK_YARN_USER_ENV"] = ",".join(k + '=' + v for (k, v) in env_map.items()) - -def parse_env(env_str): - # Turns a comma-separated of env settings into a dict that maps env vars to - # their values. - env = {} - for var_str in env_str.split(","): - parts = var_str.split("=") - if len(parts) == 2: - env[parts[0]] = parts[1] - elif len(var_str) > 0: - print "Invalid entry in SPARK_YARN_USER_ENV: " + var_str - sys.exit(1) - - return env + return gateway diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 8cf9d9cf1bd66..64f2eeb12b4fc 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -22,11 +22,14 @@ from fileinput import input from glob import glob import os +import re import shutil +import subprocess import sys -from tempfile import NamedTemporaryFile +import tempfile import time import unittest +import zipfile from pyspark.context import SparkContext from pyspark.files import SparkFiles @@ -55,7 +58,7 @@ class TestCheckpoint(PySparkTestCase): def setUp(self): PySparkTestCase.setUp(self) - self.checkpointDir = NamedTemporaryFile(delete=False) + self.checkpointDir = tempfile.NamedTemporaryFile(delete=False) os.unlink(self.checkpointDir.name) self.sc.setCheckpointDir(self.checkpointDir.name) @@ -148,7 +151,7 @@ def test_save_as_textfile_with_unicode(self): # Regression test for SPARK-970 x = u"\u00A1Hola, mundo!" data = self.sc.parallelize([x]) - tempFile = NamedTemporaryFile(delete=True) + tempFile = tempfile.NamedTemporaryFile(delete=True) tempFile.close() data.saveAsTextFile(tempFile.name) raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*"))) @@ -172,7 +175,7 @@ def test_cartesian_on_textfile(self): def test_deleting_input_files(self): # Regression test for SPARK-1025 - tempFile = NamedTemporaryFile(delete=False) + tempFile = tempfile.NamedTemporaryFile(delete=False) tempFile.write("Hello World!") tempFile.close() data = self.sc.textFile(tempFile.name) @@ -236,5 +239,125 @@ def test_termination_sigterm(self): from signal import SIGTERM self.do_termination_test(lambda daemon: os.kill(daemon.pid, SIGTERM)) + +class TestSparkSubmit(unittest.TestCase): + def setUp(self): + self.programDir = tempfile.mkdtemp() + self.sparkSubmit = os.path.join(os.environ.get("SPARK_HOME"), "bin", "spark-submit") + + def tearDown(self): + shutil.rmtree(self.programDir) + + def createTempFile(self, name, content): + """ + Create a temp file with the given name and content and return its path. + Strips leading spaces from content up to the first '|' in each line. + """ + pattern = re.compile(r'^ *\|', re.MULTILINE) + content = re.sub(pattern, '', content.strip()) + path = os.path.join(self.programDir, name) + with open(path, "w") as f: + f.write(content) + return path + + def createFileInZip(self, name, content): + """ + Create a zip archive containing a file with the given content and return its path. + Strips leading spaces from content up to the first '|' in each line. + """ + pattern = re.compile(r'^ *\|', re.MULTILINE) + content = re.sub(pattern, '', content.strip()) + path = os.path.join(self.programDir, name + ".zip") + with zipfile.ZipFile(path, 'w') as zip: + zip.writestr(name, content) + return path + + def test_single_script(self): + """Submit and test a single script file""" + script = self.createTempFile("test.py", """ + |from pyspark import SparkContext + | + |sc = SparkContext() + |print sc.parallelize([1, 2, 3]).map(lambda x: x * 2).collect() + """) + proc = subprocess.Popen([self.sparkSubmit, script], stdout=subprocess.PIPE) + out, err = proc.communicate() + self.assertEqual(0, proc.returncode) + self.assertIn("[2, 4, 6]", out) + + def test_script_with_local_functions(self): + """Submit and test a single script file calling a global function""" + script = self.createTempFile("test.py", """ + |from pyspark import SparkContext + | + |def foo(x): + | return x * 3 + | + |sc = SparkContext() + |print sc.parallelize([1, 2, 3]).map(foo).collect() + """) + proc = subprocess.Popen([self.sparkSubmit, script], stdout=subprocess.PIPE) + out, err = proc.communicate() + self.assertEqual(0, proc.returncode) + self.assertIn("[3, 6, 9]", out) + + def test_module_dependency(self): + """Submit and test a script with a dependency on another module""" + script = self.createTempFile("test.py", """ + |from pyspark import SparkContext + |from mylib import myfunc + | + |sc = SparkContext() + |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + """) + zip = self.createFileInZip("mylib.py", """ + |def myfunc(x): + | return x + 1 + """) + proc = subprocess.Popen([self.sparkSubmit, "--py-files", zip, script], + stdout=subprocess.PIPE) + out, err = proc.communicate() + self.assertEqual(0, proc.returncode) + self.assertIn("[2, 3, 4]", out) + + def test_module_dependency_on_cluster(self): + """Submit and test a script with a dependency on another module on a cluster""" + script = self.createTempFile("test.py", """ + |from pyspark import SparkContext + |from mylib import myfunc + | + |sc = SparkContext() + |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + """) + zip = self.createFileInZip("mylib.py", """ + |def myfunc(x): + | return x + 1 + """) + proc = subprocess.Popen( + [self.sparkSubmit, "--py-files", zip, "--master", "local-cluster[1,1,512]", script], + stdout=subprocess.PIPE) + out, err = proc.communicate() + self.assertEqual(0, proc.returncode) + self.assertIn("[2, 3, 4]", out) + + def test_single_script_on_cluster(self): + """Submit and test a single script on a cluster""" + script = self.createTempFile("test.py", """ + |from pyspark import SparkContext + | + |def foo(x): + | return x * 2 + | + |sc = SparkContext() + |print sc.parallelize([1, 2, 3]).map(foo).collect() + """) + proc = subprocess.Popen( + [self.sparkSubmit, "--master", "local-cluster[1,1,512]", script], + stdout=subprocess.PIPE) + out, err = proc.communicate() + self.assertEqual(0, proc.returncode) + self.assertIn("[2, 4, 6]", out) + + if __name__ == "__main__": unittest.main() diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index e33f4f9803054..566d96e16ed83 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -45,8 +45,7 @@ class ReplSuite extends FunSuite { } val interp = new SparkILoop(in, new PrintWriter(out), master) org.apache.spark.repl.Main.interp = interp - val separator = System.getProperty("path.separator") - interp.process(Array("-classpath", paths.mkString(separator))) + interp.process(Array("-classpath", paths.mkString(File.pathSeparator))) org.apache.spark.repl.Main.interp = null if (interp.sparkContext != null) { interp.sparkContext.stop() From 39b8b1489ff92697e4aeec997cdc436c7079d6f8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 6 May 2014 15:41:46 -0700 Subject: [PATCH 261/641] SPARK-1737: Warn rather than fail when Java 7+ is used to create distributions Also moves a few lines of code around in make-distribution.sh. Author: Patrick Wendell Closes #669 from pwendell/make-distribution and squashes the following commits: 8bfac49 [Patrick Wendell] Small fix 46918ec [Patrick Wendell] SPARK-1737: Warn rather than fail when Java 7+ is used to create distributions. --- make-distribution.sh | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index 8a63133bc45d4..ebcd8c74fc5a6 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -39,18 +39,11 @@ # 5) ./bin/spark-shell --master spark://my-master-ip:7077 # +set -o pipefail # Figure out where the Spark framework is installed FWDIR="$(cd `dirname $0`; pwd)" DISTDIR="$FWDIR/dist" -set -o pipefail -VERSION=$(mvn help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1) -if [ $? != 0 ]; then - echo -e "You need Maven installed to build Spark." - echo -e "Download Maven from https://maven.apache.org/" - exit -1; -fi - if [ -z "$JAVA_HOME" ]; then echo "Error: JAVA_HOME is not set, cannot proceed." exit -1 @@ -59,10 +52,17 @@ fi JAVA_CMD="$JAVA_HOME"/bin/java JAVA_VERSION=$("$JAVA_CMD" -version 2>&1) if ! [[ "$JAVA_VERSION" =~ "1.6" ]]; then - echo "Error: JAVA_HOME must point to a JDK 6 installation (see SPARK-1703)." + echo "***NOTE***: JAVA_HOME is not set to a JDK 6 installation. The resulting" + echo " distribution will not support Java 6. See SPARK-1703." echo "Output from 'java -version' was:" echo "$JAVA_VERSION" - exit -1 +fi + +VERSION=$(mvn help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1) +if [ $? != 0 ]; then + echo -e "You need Maven installed to build Spark." + echo -e "Download Maven from https://maven.apache.org/" + exit -1; fi # Initialize defaults From a000b5c3b0438c17e9973df4832c320210c29c27 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Tue, 6 May 2014 17:27:52 -0700 Subject: [PATCH 262/641] SPARK-1637: Clean up examples for 1.0 - [x] Move all of them into subpackages of org.apache.spark.examples (right now some are in org.apache.spark.streaming.examples, for instance, and others are in org.apache.spark.examples.mllib) - [x] Move Python examples into examples/src/main/python - [x] Update docs to reflect these changes Author: Sandeep This patch had conflicts when merged, resolved by Committer: Matei Zaharia Closes #571 from techaddict/SPARK-1637 and squashes the following commits: 47ef86c [Sandeep] Changes based on Discussions on PR, removing use of RawTextHelper from examples 8ed2d3f [Sandeep] Docs Updated for changes, Change for java examples 5f96121 [Sandeep] Move Python examples into examples/src/main/python 0a8dd77 [Sandeep] Move all Scala Examples to org.apache.spark.examples (some are in org.apache.spark.streaming.examples, for instance, and others are in org.apache.spark.examples.mllib) --- docs/index.md | 4 ++-- docs/python-programming-guide.md | 4 ++-- docs/streaming-programming-guide.md | 23 +++++++++---------- .../examples => examples/mllib}/JavaALS.java | 2 +- .../mllib}/JavaKMeans.java | 2 +- .../examples => examples/mllib}/JavaLR.java | 2 +- .../streaming}/JavaFlumeEventCount.java | 3 ++- .../streaming}/JavaKafkaWordCount.java | 5 ++-- .../streaming}/JavaNetworkWordCount.java | 5 ++-- .../streaming}/JavaQueueStream.java | 3 ++- .../src/main/python}/als.py | 0 .../src/main/python}/kmeans.py | 2 +- .../src/main/python}/logistic_regression.py | 2 +- .../src/main/python}/mllib/kmeans.py | 0 .../main/python}/mllib/logistic_regression.py | 0 .../src/main/python}/pagerank.py | 0 .../src/main/python}/pi.py | 0 .../src/main/python}/sort.py | 0 .../src/main/python}/transitive_closure.py | 0 .../src/main/python}/wordcount.py | 0 .../sql}/RDDRelation.scala | 2 +- .../sql/hive}/HiveFromSpark.scala | 2 +- .../streaming}/ActorWordCount.scala | 8 +++---- .../streaming}/FlumeEventCount.scala | 2 +- .../streaming}/HdfsWordCount.scala | 4 ++-- .../streaming}/KafkaWordCount.scala | 7 +++--- .../streaming}/MQTTWordCount.scala | 6 ++--- .../streaming}/NetworkWordCount.scala | 4 ++-- .../streaming}/QueueStream.scala | 2 +- .../streaming}/RawNetworkGrep.scala | 6 +---- .../RecoverableNetworkWordCount.scala | 6 ++--- .../streaming}/StatefulNetworkWordCount.scala | 4 ++-- .../streaming}/StreamingExamples.scala | 2 +- .../streaming}/TwitterAlgebirdCMS.scala | 2 +- .../streaming}/TwitterAlgebirdHLL.scala | 2 +- .../streaming}/TwitterPopularTags.scala | 2 +- .../streaming}/ZeroMQWordCount.scala | 6 ++--- .../clickstream/PageViewGenerator.scala | 6 ++--- .../clickstream/PageViewStream.scala | 8 +++---- .../spark/streaming/util/RawTextHelper.scala | 3 +-- 40 files changed, 69 insertions(+), 72 deletions(-) rename examples/src/main/java/org/apache/spark/{mllib/examples => examples/mllib}/JavaALS.java (98%) rename examples/src/main/java/org/apache/spark/{mllib/examples => examples/mllib}/JavaKMeans.java (98%) rename examples/src/main/java/org/apache/spark/{mllib/examples => examples/mllib}/JavaLR.java (98%) rename examples/src/main/java/org/apache/spark/{streaming/examples => examples/streaming}/JavaFlumeEventCount.java (96%) rename examples/src/main/java/org/apache/spark/{streaming/examples => examples/streaming}/JavaKafkaWordCount.java (96%) rename examples/src/main/java/org/apache/spark/{streaming/examples => examples/streaming}/JavaNetworkWordCount.java (95%) rename examples/src/main/java/org/apache/spark/{streaming/examples => examples/streaming}/JavaQueueStream.java (96%) rename {python/examples => examples/src/main/python}/als.py (100%) rename {python/examples => examples/src/main/python}/kmeans.py (98%) rename {python/examples => examples/src/main/python}/logistic_regression.py (97%) rename {python/examples => examples/src/main/python}/mllib/kmeans.py (100%) rename {python/examples => examples/src/main/python}/mllib/logistic_regression.py (100%) rename {python/examples => examples/src/main/python}/pagerank.py (100%) rename {python/examples => examples/src/main/python}/pi.py (100%) rename {python/examples => examples/src/main/python}/sort.py (100%) rename {python/examples => examples/src/main/python}/transitive_closure.py (100%) rename {python/examples => examples/src/main/python}/wordcount.py (100%) rename examples/src/main/scala/org/apache/spark/{sql/examples => examples/sql}/RDDRelation.scala (98%) rename examples/src/main/scala/org/apache/spark/{sql/examples => examples/sql/hive}/HiveFromSpark.scala (98%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/ActorWordCount.scala (95%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/FlumeEventCount.scala (98%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/HdfsWordCount.scala (95%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/KafkaWordCount.scala (93%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/MQTTWordCount.scala (95%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/NetworkWordCount.scala (95%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/QueueStream.scala (97%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/RawNetworkGrep.scala (92%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/RecoverableNetworkWordCount.scala (96%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/StatefulNetworkWordCount.scala (96%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/StreamingExamples.scala (97%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/TwitterAlgebirdCMS.scala (99%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/TwitterAlgebirdHLL.scala (98%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/TwitterPopularTags.scala (98%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/ZeroMQWordCount.scala (95%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/clickstream/PageViewGenerator.scala (95%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/clickstream/PageViewStream.scala (94%) diff --git a/docs/index.md b/docs/index.md index e3647717a1f18..a2f1a84371ff4 100644 --- a/docs/index.md +++ b/docs/index.md @@ -24,11 +24,11 @@ right version of Scala from [scala-lang.org](http://www.scala-lang.org/download/ # Running the Examples and Shell -Spark comes with several sample programs. Scala and Java examples are in the `examples` directory, and Python examples are in `python/examples`. +Spark comes with several sample programs. Scala, Java and Python examples are in the `examples/src/main` directory. To run one of the Java or Scala sample programs, use `./bin/run-example ` in the top-level Spark directory (the `bin/run-example` script sets up the appropriate paths and launches that program). For example, try `./bin/run-example org.apache.spark.examples.SparkPi local`. -To run a Python sample program, use `./bin/pyspark `. For example, try `./bin/pyspark ./python/examples/pi.py local`. +To run a Python sample program, use `./bin/pyspark `. For example, try `./bin/pyspark ./examples/src/main/python/pi.py local`. Each example prints usage help when run with no parameters. diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 8ea22e15a4b69..6813963bb080c 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -161,9 +161,9 @@ some example applications. # Where to Go from Here -PySpark also includes several sample programs in the [`python/examples` folder](https://github.com/apache/spark/tree/master/python/examples). +PySpark also includes several sample programs in the [`examples/src/main/python` folder](https://github.com/apache/spark/tree/master/examples/src/main/python). You can run them by passing the files to `pyspark`; e.g.: - ./bin/spark-submit python/examples/wordcount.py + ./bin/spark-submit examples/src/main/python/wordcount.py Each program prints usage help when run without arguments. diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index e8b718b303560..939599aa6855b 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -129,7 +129,7 @@ ssc.awaitTermination() // Wait for the computation to terminate {% endhighlight %} The complete code can be found in the Spark Streaming example -[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala). +[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala).
      @@ -215,7 +215,7 @@ jssc.awaitTermination(); // Wait for the computation to terminate {% endhighlight %} The complete code can be found in the Spark Streaming example -[JavaNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java). +[JavaNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java).
      @@ -234,12 +234,12 @@ Then, in a different terminal, you can start the example by using
      {% highlight bash %} -$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999 +$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount local[2] localhost 9999 {% endhighlight %}
      {% highlight bash %} -$ ./bin/run-example org.apache.spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999 +$ ./bin/run-example org.apache.spark.examples.streaming.JavaNetworkWordCount local[2] localhost 9999 {% endhighlight %}
      @@ -268,7 +268,7 @@ hello world {% highlight bash %} # TERMINAL 2: RUNNING NetworkWordCount or JavaNetworkWordCount -$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999 +$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount local[2] localhost 9999 ... ------------------------------------------- Time: 1357008430000 ms @@ -609,7 +609,7 @@ JavaPairDStream runningCounts = pairs.updateStateByKey(updateFu The update function will be called for each word, with `newValues` having a sequence of 1's (from the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete Scala code, take a look at the example -[StatefulNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala). +[StatefulNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala).

      Transform Operation

      @@ -1135,7 +1135,7 @@ If the `checkpointDirectory` exists, then the context will be recreated from the If the directory does not exist (i.e., running for the first time), then the function `functionToCreateContext` will be called to create a new context and set up the DStreams. See the Scala example -[RecoverableNetworkWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala). +[RecoverableNetworkWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala). This example appends the word counts of network data into a file. You can also explicitly create a `StreamingContext` from the checkpoint data and start the @@ -1174,7 +1174,7 @@ If the `checkpointDirectory` exists, then the context will be recreated from the If the directory does not exist (i.e., running for the first time), then the function `contextFactory` will be called to create a new context and set up the DStreams. See the Scala example -[JavaRecoverableWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples/JavaRecoverableWordCount.scala) +[JavaRecoverableWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/JavaRecoverableWordCount.scala) (note that this example is missing in the 0.9 release, so you can test it using the master branch). This example appends the word counts of network data into a file. @@ -1374,7 +1374,6 @@ package and renamed for better clarity. [ZeroMQUtils](api/java/org/apache/spark/streaming/zeromq/ZeroMQUtils.html), and [MQTTUtils](api/java/org/apache/spark/streaming/mqtt/MQTTUtils.html) -* More examples in [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples) - and [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/streaming/examples) -* [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) and -[video](http://youtu.be/g171ndOHgJ0) describing Spark Streaming. +* More examples in [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming) + and [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/streaming) +* [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) and [video](http://youtu.be/g171ndOHgJ0) describing Spark Streaming. diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java similarity index 98% rename from examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java rename to examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java index c516199d61c72..4533c4c5f241a 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.examples; +package org.apache.spark.examples.mllib; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java similarity index 98% rename from examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java rename to examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java index 7461609ab9e8f..0cfb8e69ed28f 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.examples; +package org.apache.spark.examples.mllib; import java.util.regex.Pattern; diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLR.java similarity index 98% rename from examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java rename to examples/src/main/java/org/apache/spark/examples/mllib/JavaLR.java index e3ab87cc722f3..f6e48b498727b 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLR.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.examples; +package org.apache.spark.examples.mllib; import java.util.regex.Pattern; diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java similarity index 96% rename from examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java index c59f7538f8a82..a5ece68cef870 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples; +package org.apache.spark.examples.streaming; import org.apache.spark.api.java.function.Function; +import org.apache.spark.examples.streaming.StreamingExamples; import org.apache.spark.streaming.*; import org.apache.spark.streaming.api.java.*; import org.apache.spark.streaming.flume.FlumeUtils; diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java similarity index 96% rename from examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java index 8da9bcd05ac38..da51eb189a649 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples; +package org.apache.spark.examples.streaming; import java.util.Map; import java.util.HashMap; @@ -26,6 +26,7 @@ import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.examples.streaming.StreamingExamples; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; @@ -44,7 +45,7 @@ * is the number of threads the kafka consumer should use * * Example: - * `./bin/run-example org.apache.spark.streaming.examples.JavaKafkaWordCount local[2] zoo01,zoo02, + * `./bin/run-example org.apache.spark.examples.streaming.JavaKafkaWordCount local[2] zoo01,zoo02, * zoo03 my-consumer-group topic1,topic2 1` */ diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java similarity index 95% rename from examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java index 098c329ff6808..ac84991d87b8b 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples; +package org.apache.spark.examples.streaming; import com.google.common.collect.Lists; import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; @@ -23,6 +23,7 @@ import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.examples.streaming.StreamingExamples; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; @@ -39,7 +40,7 @@ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run org.apache.spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999` + * `$ ./run org.apache.spark.examples.streaming.JavaNetworkWordCount local[2] localhost 9999` */ public final class JavaNetworkWordCount { private static final Pattern SPACE = Pattern.compile(" "); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java similarity index 96% rename from examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java index 88ad341641e0a..819311968fac5 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java @@ -15,13 +15,14 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples; +package org.apache.spark.examples.streaming; import com.google.common.collect.Lists; import scala.Tuple2; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.examples.streaming.StreamingExamples; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; diff --git a/python/examples/als.py b/examples/src/main/python/als.py similarity index 100% rename from python/examples/als.py rename to examples/src/main/python/als.py diff --git a/python/examples/kmeans.py b/examples/src/main/python/kmeans.py similarity index 98% rename from python/examples/kmeans.py rename to examples/src/main/python/kmeans.py index d8387b0b183e6..e3596488faf9e 100755 --- a/python/examples/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -18,7 +18,7 @@ """ The K-means algorithm written from scratch against PySpark. In practice, one may prefer to use the KMeans algorithm in MLlib, as shown in -python/examples/mllib/kmeans.py. +examples/src/main/python/mllib/kmeans.py. This example requires NumPy (http://www.numpy.org/). """ diff --git a/python/examples/logistic_regression.py b/examples/src/main/python/logistic_regression.py similarity index 97% rename from python/examples/logistic_regression.py rename to examples/src/main/python/logistic_regression.py index 28d52e6a40b45..fe5373cf799b1 100755 --- a/python/examples/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -20,7 +20,7 @@ to act on batches of input data using efficient matrix operations. In practice, one may prefer to use the LogisticRegression algorithm in -MLlib, as shown in python/examples/mllib/logistic_regression.py. +MLlib, as shown in examples/src/main/python/mllib/logistic_regression.py. """ from collections import namedtuple diff --git a/python/examples/mllib/kmeans.py b/examples/src/main/python/mllib/kmeans.py similarity index 100% rename from python/examples/mllib/kmeans.py rename to examples/src/main/python/mllib/kmeans.py diff --git a/python/examples/mllib/logistic_regression.py b/examples/src/main/python/mllib/logistic_regression.py similarity index 100% rename from python/examples/mllib/logistic_regression.py rename to examples/src/main/python/mllib/logistic_regression.py diff --git a/python/examples/pagerank.py b/examples/src/main/python/pagerank.py similarity index 100% rename from python/examples/pagerank.py rename to examples/src/main/python/pagerank.py diff --git a/python/examples/pi.py b/examples/src/main/python/pi.py similarity index 100% rename from python/examples/pi.py rename to examples/src/main/python/pi.py diff --git a/python/examples/sort.py b/examples/src/main/python/sort.py similarity index 100% rename from python/examples/sort.py rename to examples/src/main/python/sort.py diff --git a/python/examples/transitive_closure.py b/examples/src/main/python/transitive_closure.py similarity index 100% rename from python/examples/transitive_closure.py rename to examples/src/main/python/transitive_closure.py diff --git a/python/examples/wordcount.py b/examples/src/main/python/wordcount.py similarity index 100% rename from python/examples/wordcount.py rename to examples/src/main/python/wordcount.py diff --git a/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala similarity index 98% rename from examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala rename to examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index 8210ad977f066..ff9254b044c24 100644 --- a/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.examples +package org.apache.spark.examples.sql import org.apache.spark.SparkContext import org.apache.spark.sql.SQLContext diff --git a/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala similarity index 98% rename from examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala rename to examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 62329bde84481..66ce93a26ef42 100644 --- a/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.hive.examples +package org.apache.spark.examples.sql.hive import org.apache.spark.SparkContext import org.apache.spark.sql._ diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala similarity index 95% rename from examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala index c845dd8904c90..84cf43df0f96c 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import scala.collection.mutable.LinkedList import scala.reflect.ClassTag @@ -78,7 +78,7 @@ class FeederActor extends Actor { * goes and subscribe to a typical publisher/feeder actor and receives * data. * - * @see [[org.apache.spark.streaming.examples.FeederActor]] + * @see [[org.apache.spark.examples.streaming.FeederActor]] */ class SampleActorReceiver[T: ClassTag](urlOfPublisher: String) extends Actor with ActorHelper { @@ -131,9 +131,9 @@ object FeederActor { * and describe the AkkaSystem that Spark Sample feeder is running on. * * To run this example locally, you may run Feeder Actor as - * `$ ./bin/run-example org.apache.spark.streaming.examples.FeederActor 127.0.1.1 9999` + * `$ ./bin/run-example org.apache.spark.examples.streaming.FeederActor 127.0.1.1 9999` * and then run the example - * `./bin/run-example org.apache.spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999` + * `./bin/run-example org.apache.spark.examples.streaming.ActorWordCount local[2] 127.0.1.1 9999` */ object ActorWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala similarity index 98% rename from examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala index 26b6024534124..5b2a1035fc779 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala similarity index 95% rename from examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala index 7f86fc792aacf..b440956ba3137 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ @@ -27,7 +27,7 @@ import org.apache.spark.streaming.StreamingContext._ * is the directory that Spark Streaming will use to find and read new text files. * * To run this on your local machine on directory `localdir`, run this example - * `$ ./bin/run-example org.apache.spark.streaming.examples.HdfsWordCount local[2] localdir` + * `$ ./bin/run-example org.apache.spark.examples.streaming.HdfsWordCount local[2] localdir` * Then create a text file in `localdir` and the words in the file will get counted. */ object HdfsWordCount { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala similarity index 93% rename from examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index 2aa4f1474a59e..c3aae5af05b1c 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import java.util.Properties @@ -24,7 +24,6 @@ import kafka.producer._ import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.kafka._ -import org.apache.spark.streaming.util.RawTextHelper._ // scalastyle:off /** @@ -37,7 +36,7 @@ import org.apache.spark.streaming.util.RawTextHelper._ * is the number of threads the kafka consumer should use * * Example: - * `./bin/run-example org.apache.spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` + * `./bin/run-example org.apache.spark.examples.streaming.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` */ // scalastyle:on object KafkaWordCount { @@ -59,7 +58,7 @@ object KafkaWordCount { val lines = KafkaUtils.createStream(ssc, zkQuorum, group, topicpMap).map(_._2) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1L)) - .reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2) + .reduceByKeyAndWindow(_ + _, _ - _, Minutes(10), Seconds(2), 2) wordCounts.print() ssc.start() diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala similarity index 95% rename from examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala index 62aef0fb47107..47bf1e5a06439 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.eclipse.paho.client.mqttv3.{MqttClient, MqttClientPersistence, MqttException, MqttMessage, MqttTopic} import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence @@ -79,9 +79,9 @@ object MQTTPublisher { * and describe where Mqtt publisher is running. * * To run this example locally, you may run publisher as - * `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTPublisher tcp://localhost:1883 foo` + * `$ ./bin/run-example org.apache.spark.examples.streaming.MQTTPublisher tcp://localhost:1883 foo` * and run the example as - * `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTWordCount local[2] tcp://localhost:1883 foo` + * `$ ./bin/run-example org.apache.spark.examples.streaming.MQTTWordCount local[2] tcp://localhost:1883 foo` */ // scalastyle:on object MQTTWordCount { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala similarity index 95% rename from examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala index 272ab11212451..acfe9a4da3596 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ @@ -32,7 +32,7 @@ import org.apache.spark.storage.StorageLevel * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999` + * `$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount local[2] localhost 9999` */ // scalastyle:on object NetworkWordCount { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala similarity index 97% rename from examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala index ff2a205ec1c15..f92f72f2de876 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import scala.collection.mutable.SynchronizedQueue diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala similarity index 92% rename from examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala index d915c0c39b334..1b0319a046433 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ -import org.apache.spark.streaming.util.RawTextHelper import org.apache.spark.util.IntParam /** @@ -52,9 +51,6 @@ object RawNetworkGrep { val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) - // Warm up the JVMs on master and slave for JIT compilation to kick in - RawTextHelper.warmUp(ssc.sparkContext) - val rawStreams = (1 to numStreams).map(_ => ssc.rawSocketStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray val union = ssc.union(rawStreams) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala similarity index 96% rename from examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index 4aacbb1991418..b0bc31cc66ab5 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.apache.spark.streaming.{Time, Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ @@ -44,7 +44,7 @@ import java.nio.charset.Charset * * and run the example as * - * `$ ./run-example org.apache.spark.streaming.examples.RecoverableNetworkWordCount \ + * `$ ./run-example org.apache.spark.examples.streaming.RecoverableNetworkWordCount \ * local[2] localhost 9999 ~/checkpoint/ ~/out` * * If the directory ~/checkpoint/ does not exist (e.g. running for the first time), it will create @@ -56,7 +56,7 @@ import java.nio.charset.Charset * * `$ ./spark-class org.apache.spark.deploy.Client -s launch \ * \ - * org.apache.spark.streaming.examples.RecoverableNetworkWordCount \ + * org.apache.spark.examples.streaming.RecoverableNetworkWordCount \ * localhost 9999 ~/checkpoint ~/out` * * would typically be diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala similarity index 96% rename from examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala index ef94c9298dd93..8001d56c98d86 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ @@ -31,7 +31,7 @@ import org.apache.spark.streaming.StreamingContext._ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./bin/run-example org.apache.spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` + * `$ ./bin/run-example org.apache.spark.examples.streaming.StatefulNetworkWordCount local[2] localhost 9999` */ // scalastyle:on object StatefulNetworkWordCount { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala similarity index 97% rename from examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala index 99f1502046f53..8396e65d0d588 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.apache.spark.Logging diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala similarity index 99% rename from examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala index c38905e8f3663..b12617d881787 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import com.twitter.algebird._ diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala similarity index 98% rename from examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala index c067046f9020e..22f232c72545c 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import com.twitter.algebird.HyperLogLogMonoid import com.twitter.algebird.HyperLogLog._ diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala similarity index 98% rename from examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala index 2597c8178862a..5b58e94600a16 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.apache.spark.streaming.{Seconds, StreamingContext} import StreamingContext._ diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala similarity index 95% rename from examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala index 109ff855b5228..de46e5f5b10b6 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import akka.actor.ActorSystem import akka.actor.actorRef2Scala @@ -68,9 +68,9 @@ object SimpleZeroMQPublisher { * and describe where zeroMq publisher is running. * * To run this example locally, you may run publisher as - * `$ ./bin/run-example org.apache.spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` + * `$ ./bin/run-example org.apache.spark.examples.streaming.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` * and run the example as - * `$ ./bin/run-example org.apache.spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` + * `$ ./bin/run-example org.apache.spark.examples.streaming.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` */ // scalastyle:on object ZeroMQWordCount { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala similarity index 95% rename from examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala index 251f65fe4df9c..97e0cb92078dc 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples.clickstream +package org.apache.spark.examples.streaming.clickstream import java.net.ServerSocket import java.io.PrintWriter @@ -40,8 +40,8 @@ object PageView extends Serializable { /** Generates streaming events to simulate page views on a website. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewGenerator 44444 10 - * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10 + * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444 * * When running this, you may want to set the root logging level to ERROR in * conf/log4j.properties to reduce the verbosity of the output. diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala similarity index 94% rename from examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala index 673013f7cf948..d30ceffbe29cb 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala @@ -15,19 +15,19 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples.clickstream +package org.apache.spark.examples.streaming.clickstream import org.apache.spark.SparkContext._ import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ -import org.apache.spark.streaming.examples.StreamingExamples +import org.apache.spark.examples.streaming.StreamingExamples // scalastyle:off /** Analyses a streaming dataset of web page views. This class demonstrates several types of * operators available in Spark streaming. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewGenerator 44444 10 - * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10 + * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444 */ // scalastyle:on object PageViewStream { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala index bbf57ef9275c0..a73d6f3bf0661 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala @@ -25,7 +25,7 @@ import scala.collection.JavaConversions.mapAsScalaMap private[streaming] object RawTextHelper { - /** + /** * Splits lines and counts the words. */ def splitAndCountPartitions(iter: Iterator[String]): Iterator[(String, Long)] = { @@ -114,4 +114,3 @@ object RawTextHelper { def max(v1: Long, v2: Long) = math.max(v1, v2) } - From 25ad8f93012730115a8a1fac649fe3e842c045b3 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 6 May 2014 20:07:22 -0700 Subject: [PATCH 263/641] SPARK-1727. Correct small compile errors, typos, and markdown issues in (primarly) MLlib docs While play-testing the Scala and Java code examples in the MLlib docs, I noticed a number of small compile errors, and some typos. This led to finding and fixing a few similar items in other docs. Then in the course of building the site docs to check the result, I found a few small suggestions for the build instructions. I also found a few more formatting and markdown issues uncovered when I accidentally used maruku instead of kramdown. Author: Sean Owen Closes #653 from srowen/SPARK-1727 and squashes the following commits: 6e7c38a [Sean Owen] Final doc updates - one more compile error, and use of mean instead of sum and count 8f5e847 [Sean Owen] Fix markdown syntax issues that maruku flags, even though we use kramdown (but only those that do not affect kramdown's output) 99966a9 [Sean Owen] Update issue tracker URL in docs 23c9ac3 [Sean Owen] Add Scala Naive Bayes example, to use existing example data file (whose format needed a tweak) 8c81982 [Sean Owen] Fix small compile errors and typos across MLlib docs --- docs/README.md | 9 ++--- docs/_config.yml | 2 +- docs/bagel-programming-guide.md | 2 +- docs/cluster-overview.md | 2 +- docs/configuration.md | 10 +++--- docs/java-programming-guide.md | 20 +++++------ docs/mllib-basics.md | 14 +++++--- docs/mllib-clustering.md | 4 +-- docs/mllib-collaborative-filtering.md | 2 +- docs/mllib-decision-tree.md | 8 ++--- docs/mllib-dimensionality-reduction.md | 7 ++++ docs/mllib-guide.md | 2 +- docs/mllib-linear-methods.md | 13 +++---- docs/mllib-naive-bayes.md | 48 +++++++++++++++++--------- docs/scala-programming-guide.md | 9 +++-- docs/sql-programming-guide.md | 1 + mllib/data/sample_naive_bayes_data.txt | 12 +++---- 17 files changed, 97 insertions(+), 68 deletions(-) diff --git a/docs/README.md b/docs/README.md index 75b1811ba99af..f1eb644f93406 100644 --- a/docs/README.md +++ b/docs/README.md @@ -14,9 +14,10 @@ The markdown code can be compiled to HTML using the [Jekyll tool](http://jekyllrb.com). To use the `jekyll` command, you will need to have Jekyll installed. The easiest way to do this is via a Ruby Gem, see the -[jekyll installation instructions](http://jekyllrb.com/docs/installation). -Compiling the site with Jekyll will create a directory called -_site containing index.html as well as the rest of the compiled files. +[jekyll installation instructions](http://jekyllrb.com/docs/installation). +If not already installed, you need to install `kramdown` with `sudo gem install kramdown`. +Execute `jekyll` from the `docs/` directory. Compiling the site with Jekyll will create a directory called +`_site` containing index.html as well as the rest of the compiled files. You can modify the default Jekyll build as follows: @@ -44,6 +45,6 @@ You can build just the Spark scaladoc by running `sbt/sbt doc` from the SPARK_PR Similarly, you can build just the PySpark epydoc by running `epydoc --config epydoc.conf` from the SPARK_PROJECT_ROOT/pyspark directory. Documentation is only generated for classes that are listed as public in `__init__.py`. -When you run `jekyll` in the docs directory, it will also copy over the scaladoc for the various Spark subprojects into the docs directory (and then also into the _site directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/). +When you run `jekyll` in the `docs` directory, it will also copy over the scaladoc for the various Spark subprojects into the `docs` directory (and then also into the `_site` directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/). NOTE: To skip the step of building and copying over the Scala and Python API docs, run `SKIP_API=1 jekyll`. diff --git a/docs/_config.yml b/docs/_config.yml index d585b8c5ea763..d177e38f884ca 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -8,5 +8,5 @@ SPARK_VERSION_SHORT: 1.0.0 SCALA_BINARY_VERSION: "2.10" SCALA_VERSION: "2.10.4" MESOS_VERSION: 0.13.0 -SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net +SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md index da6d0c9dcd97b..14f43cb6d3946 100644 --- a/docs/bagel-programming-guide.md +++ b/docs/bagel-programming-guide.md @@ -46,7 +46,7 @@ import org.apache.spark.bagel.Bagel._ Next, we load a sample graph from a text file as a distributed dataset and package it into `PRVertex` objects. We also cache the distributed dataset because Bagel will use it multiple times and we'd like to avoid recomputing it. {% highlight scala %} -val input = sc.textFile("pagerank_data.txt") +val input = sc.textFile("data/pagerank_data.txt") val numVerts = input.count() diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 79b0061e2c5a1..162c415b5883e 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -181,7 +181,7 @@ The following table summarizes terms you'll see used to refer to cluster concept Distinguishes where the driver process runs. In "cluster" mode, the framework launches the driver inside of the cluster. In "client" mode, the submitter launches the driver outside of the cluster. - + Worker node Any node that can run application code in the cluster diff --git a/docs/configuration.md b/docs/configuration.md index d6f316ba5f773..5b034e3cb3d47 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -26,10 +26,10 @@ application name), as well as arbitrary key-value pairs through the `set()` meth initialize an application as follows: {% highlight scala %} -val conf = new SparkConf() - .setMaster("local") - .setAppName("My application") - .set("spark.executor.memory", "1g") +val conf = new SparkConf(). + setMaster("local"). + setAppName("My application"). + set("spark.executor.memory", "1g") val sc = new SparkContext(conf) {% endhighlight %} @@ -318,7 +318,7 @@ Apart from these, the following properties are also available, and may be useful When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches objects to prevent writing redundant data, however that stops garbage collection of those objects. By calling 'reset' you flush that info from the serializer, and allow old - objects to be collected. To turn off this periodic reset set it to a value of <= 0. + objects to be collected. To turn off this periodic reset set it to a value <= 0. By default it will reset the serializer every 10,000 objects. diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md index 07c8512bf9294..c34eb28fc06a2 100644 --- a/docs/java-programming-guide.md +++ b/docs/java-programming-guide.md @@ -55,7 +55,7 @@ classes. RDD methods like `map` are overloaded by specialized `PairFunction` and `DoubleFunction` classes, allowing them to return RDDs of the appropriate types. Common methods like `filter` and `sample` are implemented by each specialized RDD class, so filtering a `PairRDD` returns a new `PairRDD`, -etc (this acheives the "same-result-type" principle used by the [Scala collections +etc (this achieves the "same-result-type" principle used by the [Scala collections framework](http://docs.scala-lang.org/overviews/core/architecture-of-scala-collections.html)). ## Function Interfaces @@ -102,7 +102,7 @@ the following changes: `Function` classes will need to use `implements` rather than `extends`. * Certain transformation functions now have multiple versions depending on the return type. In Spark core, the map functions (`map`, `flatMap`, and - `mapPartitons`) have type-specific versions, e.g. + `mapPartitions`) have type-specific versions, e.g. [`mapToPair`](api/java/org/apache/spark/api/java/JavaRDDLike.html#mapToPair(org.apache.spark.api.java.function.PairFunction)) and [`mapToDouble`](api/java/org/apache/spark/api/java/JavaRDDLike.html#mapToDouble(org.apache.spark.api.java.function.DoubleFunction)). Spark Streaming also uses the same approach, e.g. [`transformToPair`](api/java/org/apache/spark/streaming/api/java/JavaDStreamLike.html#transformToPair(org.apache.spark.api.java.function.Function)). @@ -115,11 +115,11 @@ As an example, we will implement word count using the Java API. import org.apache.spark.api.java.*; import org.apache.spark.api.java.function.*; -JavaSparkContext sc = new JavaSparkContext(...); -JavaRDD lines = ctx.textFile("hdfs://..."); +JavaSparkContext jsc = new JavaSparkContext(...); +JavaRDD lines = jsc.textFile("hdfs://..."); JavaRDD words = lines.flatMap( new FlatMapFunction() { - public Iterable call(String s) { + @Override public Iterable call(String s) { return Arrays.asList(s.split(" ")); } } @@ -140,10 +140,10 @@ Here, the `FlatMapFunction` was created inline; another option is to subclass {% highlight java %} class Split extends FlatMapFunction { - public Iterable call(String s) { + @Override public Iterable call(String s) { return Arrays.asList(s.split(" ")); } -); +} JavaRDD words = lines.flatMap(new Split()); {% endhighlight %} @@ -162,8 +162,8 @@ Continuing with the word count example, we map each word to a `(word, 1)` pair: import scala.Tuple2; JavaPairRDD ones = words.mapToPair( new PairFunction() { - public Tuple2 call(String s) { - return new Tuple2(s, 1); + @Override public Tuple2 call(String s) { + return new Tuple2(s, 1); } } ); @@ -178,7 +178,7 @@ occurrences of each word: {% highlight java %} JavaPairRDD counts = ones.reduceByKey( new Function2() { - public Integer call(Integer i1, Integer i2) { + @Override public Integer call(Integer i1, Integer i2) { return i1 + i2; } } diff --git a/docs/mllib-basics.md b/docs/mllib-basics.md index 710ce1721fe25..704308802d65b 100644 --- a/docs/mllib-basics.md +++ b/docs/mllib-basics.md @@ -9,7 +9,7 @@ title: MLlib - Basics MLlib supports local vectors and matrices stored on a single machine, as well as distributed matrices backed by one or more RDDs. In the current implementation, local vectors and matrices are simple data models -to serve public interfaces. The underly linear algebra operations are provided by +to serve public interfaces. The underlying linear algebra operations are provided by [Breeze](http://www.scalanlp.org/) and [jblas](http://jblas.org/). A training example used in supervised learning is called "labeled point" in MLlib. @@ -205,7 +205,7 @@ import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.mllib.util.MLUtils; import org.apache.spark.rdd.RDDimport; -RDD[LabeledPoint] training = MLUtils.loadLibSVMData(sc, "mllib/data/sample_libsvm_data.txt") +RDD training = MLUtils.loadLibSVMData(jsc, "mllib/data/sample_libsvm_data.txt"); {% endhighlight %} @@ -307,6 +307,7 @@ A [`RowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.R created from a `JavaRDD` instance. Then we can compute its column summary statistics. {% highlight java %} +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.distributed.RowMatrix; @@ -348,10 +349,10 @@ val mat: RowMatrix = ... // a RowMatrix val summary: MultivariateStatisticalSummary = mat.computeColumnSummaryStatistics() println(summary.mean) // a dense vector containing the mean value for each column println(summary.variance) // column-wise variance -println(summary.numNonzers) // number of nonzeros in each column +println(summary.numNonzeros) // number of nonzeros in each column // Compute the covariance matrix. -val Cov: Matrix = mat.computeCovariance() +val cov: Matrix = mat.computeCovariance() {% endhighlight %} @@ -397,11 +398,12 @@ wrapper over `(long, Vector)`. An `IndexedRowMatrix` can be converted to a `Row its row indices. {% highlight java %} +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.mllib.linalg.distributed.IndexedRow; import org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix; import org.apache.spark.mllib.linalg.distributed.RowMatrix; -JavaRDD[IndexedRow] rows = ... // a JavaRDD of indexed rows +JavaRDD rows = ... // a JavaRDD of indexed rows // Create an IndexedRowMatrix from a JavaRDD. IndexedRowMatrix mat = new IndexedRowMatrix(rows.rdd()); @@ -458,7 +460,9 @@ wrapper over `(long, long, double)`. A `CoordinateMatrix` can be converted to a with sparse rows by calling `toIndexedRowMatrix`. {% highlight scala %} +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.mllib.linalg.distributed.CoordinateMatrix; +import org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix; import org.apache.spark.mllib.linalg.distributed.MatrixEntry; JavaRDD entries = ... // a JavaRDD of matrix entries diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index b3293afe408d0..276868fa8490d 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -18,7 +18,7 @@ models are trained for each cluster). MLlib supports [k-means](http://en.wikipedia.org/wiki/K-means_clustering) clustering, one of the most commonly used clustering algorithms that clusters the data points into -predfined number of clusters. The MLlib implementation includes a parallelized +predefined number of clusters. The MLlib implementation includes a parallelized variant of the [k-means++](http://en.wikipedia.org/wiki/K-means%2B%2B) method called [kmeans||](http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf). The implementation in MLlib has the following parameters: @@ -30,7 +30,7 @@ initialization via k-means\|\|. * *runs* is the number of times to run the k-means algorithm (k-means is not guaranteed to find a globally optimal solution, and when run multiple times on a given dataset, the algorithm returns the best clustering result). -* *initializiationSteps* determines the number of steps in the k-means\|\| algorithm. +* *initializationSteps* determines the number of steps in the k-means\|\| algorithm. * *epsilon* determines the distance threshold within which we consider k-means to have converged. ## Examples diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index 79f5e3a7ca4fb..f486c56e55907 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -77,7 +77,7 @@ val ratesAndPreds = ratings.map{ }.join(predictions) val MSE = ratesAndPreds.map{ case ((user, product), (r1, r2)) => math.pow((r1- r2), 2) -}.reduce(_ + _)/ratesAndPreds.count +}.mean() println("Mean Squared Error = " + MSE) {% endhighlight %} diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 0693766990732..296277e58b341 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -83,19 +83,19 @@ Section 9.2.4 in [Elements of Statistical Machine Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for details). For example, for a binary classification problem with one categorical feature with three categories A, B and C with corresponding proportion of label 1 as 0.2, 0.6 and 0.4, the categorical -features are orded as A followed by C followed B or A, B, C. The two split candidates are A \| C, B +features are ordered as A followed by C followed B or A, B, C. The two split candidates are A \| C, B and A , B \| C where \| denotes the split. ### Stopping rule The recursive tree construction is stopped at a node when one of the two conditions is met: -1. The node depth is equal to the `maxDepth` training parammeter +1. The node depth is equal to the `maxDepth` training parameter 2. No split candidate leads to an information gain at the node. ### Practical limitations -1. The tree implementation stores an Array[Double] of size *O(#features \* #splits \* 2^maxDepth)* +1. The tree implementation stores an `Array[Double]` of size *O(#features \* #splits \* 2^maxDepth)* in memory for aggregating histograms over partitions. The current implementation might not scale to very deep trees since the memory requirement grows exponentially with tree depth. 2. The implemented algorithm reads both sparse and dense data. However, it is not optimized for @@ -178,7 +178,7 @@ val valuesAndPreds = parsedData.map { point => val prediction = model.predict(point.features) (point.label, prediction) } -val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.reduce(_ + _)/valuesAndPreds.count +val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.mean() println("training Mean Squared Error = " + MSE) {% endhighlight %} diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md index 4e9ecf7c006fa..ab24663cfe258 100644 --- a/docs/mllib-dimensionality-reduction.md +++ b/docs/mllib-dimensionality-reduction.md @@ -44,6 +44,10 @@ say, less than $1000$, but many rows, which we call *tall-and-skinny*.
      {% highlight scala %} +import org.apache.spark.mllib.linalg.Matrix +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.linalg.SingularValueDecomposition + val mat: RowMatrix = ... // Compute the top 20 singular values and corresponding singular vectors. @@ -74,6 +78,9 @@ and use them to project the vectors into a low-dimensional space. The number of columns should be small, e.g, less than 1000. {% highlight scala %} +import org.apache.spark.mllib.linalg.Matrix +import org.apache.spark.mllib.linalg.distributed.RowMatrix + val mat: RowMatrix = ... // Compute the top 10 principal components. diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index c49f857d07557..842ca5c8c6d8a 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -94,7 +94,7 @@ import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; double[] array = ... // a double array -Vector vector = Vectors.dense(array) // a dense vector +Vector vector = Vectors.dense(array); // a dense vector {% endhighlight %} [`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index ebb555f974bf7..40b7a7f80708c 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -63,7 +63,7 @@ methods MLlib supports: hinge loss$\max \{0, 1-y \wv^T \x \}, \quad y \in \{-1, +1\}$ - $\begin{cases}-y \cdot \x & \text{if $y \wv^T \x <1$}, \\ 0 & + $\begin{cases}-y \cdot \x & \text{if $y \wv^T \x <1$}, \\ 0 & \text{otherwise}.\end{cases}$ @@ -225,10 +225,11 @@ algorithm for 200 iterations. import org.apache.spark.mllib.optimization.L1Updater val svmAlg = new SVMWithSGD() -svmAlg.optimizer.setNumIterations(200) - .setRegParam(0.1) - .setUpdater(new L1Updater) -val modelL1 = svmAlg.run(parsedData) +svmAlg.optimizer. + setNumIterations(200). + setRegParam(0.1). + setUpdater(new L1Updater) +val modelL1 = svmAlg.run(training) {% endhighlight %} Similarly, you can use replace `SVMWithSGD` by @@ -322,7 +323,7 @@ val valuesAndPreds = parsedData.map { point => val prediction = model.predict(point.features) (point.label, prediction) } -val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.reduce(_ + _) / valuesAndPreds.count +val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.mean() println("training Mean Squared Error = " + MSE) {% endhighlight %} diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index 6160fe5b2fe8c..c47508b7daa2d 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -7,13 +7,13 @@ Naive Bayes is a simple multiclass classification algorithm with the assumption between every pair of features. Naive Bayes can be trained very efficiently. Within a single pass to the training data, it computes the conditional probability distribution of each feature given label, and then it applies Bayes' theorem to compute the conditional probability distribution of label -given an observation and use it for prediction. For more details, please visit the wikipedia page +given an observation and use it for prediction. For more details, please visit the Wikipedia page [Naive Bayes classifier](http://en.wikipedia.org/wiki/Naive_Bayes_classifier). In MLlib, we implemented multinomial naive Bayes, which is typically used for document classification. Within that context, each observation is a document, each feature represents a term, -whose value is the frequency of the term. For its formulation, please visit the wikipedia page -[Multinomial naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes) +whose value is the frequency of the term. For its formulation, please visit the Wikipedia page +[Multinomial Naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes) or the section [Naive Bayes text classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html) from the book Introduction to Information @@ -36,9 +36,18 @@ can be used for evaluation and prediction. {% highlight scala %} import org.apache.spark.mllib.classification.NaiveBayes - -val training: RDD[LabeledPoint] = ... // training set -val test: RDD[LabeledPoint] = ... // test set +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint + +val data = sc.textFile("mllib/data/sample_naive_bayes_data.txt") +val parsedData = data.map { line => + val parts = line.split(',') + LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) +} +// Split data into training (60%) and test (40%). +val splits = parsedData.randomSplit(Array(0.6, 0.4), seed = 11L) +val training = splits(0) +val test = splits(1) val model = NaiveBayes.train(training, lambda = 1.0) val prediction = model.predict(test.map(_.features)) @@ -58,29 +67,36 @@ optionally smoothing parameter `lambda` as input, and output a can be used for evaluation and prediction. {% highlight java %} +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.classification.NaiveBayes; +import org.apache.spark.mllib.classification.NaiveBayesModel; +import org.apache.spark.mllib.regression.LabeledPoint; +import scala.Tuple2; JavaRDD training = ... // training set JavaRDD test = ... // test set -NaiveBayesModel model = NaiveBayes.train(training.rdd(), 1.0); +final NaiveBayesModel model = NaiveBayes.train(training.rdd(), 1.0); -JavaRDD prediction = model.predict(test.map(new Function() { - public Vector call(LabeledPoint p) { - return p.features(); +JavaRDD prediction = + test.map(new Function() { + @Override public Double call(LabeledPoint p) { + return model.predict(p.features()); } - }) + }); JavaPairRDD predictionAndLabel = prediction.zip(test.map(new Function() { - public Double call(LabeledPoint p) { + @Override public Double call(LabeledPoint p) { return p.label(); } - }) + })); double accuracy = 1.0 * predictionAndLabel.filter(new Function, Boolean>() { - public Boolean call(Tuple2 pl) { + @Override public Boolean call(Tuple2 pl) { return pl._1() == pl._2(); } - }).count() / test.count() + }).count() / test.count(); {% endhighlight %}
      @@ -93,7 +109,7 @@ smoothing parameter `lambda` as input, and output a [NaiveBayesModel](api/pyspark/pyspark.mllib.classification.NaiveBayesModel-class.html), which can be used for evaluation and prediction. - + {% highlight python %} from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.classification import NaiveBayes diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index b8d89cf00ffbd..e7ceaa22c3302 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -48,12 +48,12 @@ how to access a cluster. To create a `SparkContext` you first need to build a `S that contains information about your application. {% highlight scala %} -val conf = new SparkConf().setAppName().setMaster() +val conf = new SparkConf().setAppName(appName).setMaster(master) new SparkContext(conf) {% endhighlight %} -The `` parameter is a string specifying a [Spark, Mesos or YARN cluster URL](#master-urls) -to connect to, or a special "local" string to run in local mode, as described below. `` is +The `master` parameter is a string specifying a [Spark, Mesos or YARN cluster URL](#master-urls) +to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. It's also possible to set these variables [using a configuration file](cluster-overview.html#loading-configurations-from-a-file) which avoids hard-coding the master name in your application. @@ -81,9 +81,8 @@ The master URL passed to Spark can be in one of the following formats: - - diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 0c743c9d6010d..8a785450adfde 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -416,3 +416,4 @@ results = hiveCtx.hql("FROM src SELECT key, value").collect() {% endhighlight %} + diff --git a/mllib/data/sample_naive_bayes_data.txt b/mllib/data/sample_naive_bayes_data.txt index f874adbaf4665..981da382d6ac8 100644 --- a/mllib/data/sample_naive_bayes_data.txt +++ b/mllib/data/sample_naive_bayes_data.txt @@ -1,6 +1,6 @@ -0, 1 0 0 -0, 2 0 0 -1, 0 1 0 -1, 0 2 0 -2, 0 0 1 -2, 0 0 2 +0,1 0 0 +0,2 0 0 +1,0 1 0 +1,0 2 0 +2,0 0 1 +2,0 0 2 From 48ba3b8cdc3bdc7c67bc465d1f047fa3f44d7085 Mon Sep 17 00:00:00 2001 From: Ethan Jewett Date: Tue, 6 May 2014 20:50:08 -0700 Subject: [PATCH 264/641] Proposal: clarify Scala programming guide on caching ... ... with regards to saved map output. Wording taken partially from Matei Zaharia's email to the Spark user list. http://apache-spark-user-list.1001560.n3.nabble.com/performance-improvement-on-second-operation-without-caching-td5227.html Author: Ethan Jewett Closes #668 from esjewett/Doc-update and squashes the following commits: 11793ce [Ethan Jewett] Update based on feedback 171e670 [Ethan Jewett] Clarify Scala programming guide on caching ... --- docs/scala-programming-guide.md | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index e7ceaa22c3302..f25e9cca88524 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -145,7 +145,7 @@ RDDs support two types of operations: *transformations*, which create a new data All transformations in Spark are lazy, in that they do not compute their results right away. Instead, they just remember the transformations applied to some base dataset (e.g. a file). The transformations are only computed when an action requires a result to be returned to the driver program. This design enables Spark to run more efficiently -- for example, we can realize that a dataset created through `map` will be used in a `reduce` and return only the result of the `reduce` to the driver, rather than the larger mapped dataset. -By default, each transformed RDD is recomputed each time you run an action on it. However, you may also *persist* an RDD in memory using the `persist` (or `cache`) method, in which case Spark will keep the elements around on the cluster for much faster access the next time you query it. There is also support for persisting datasets on disk, or replicated across the cluster. The next section in this document describes these options. +By default, each transformed RDD may be recomputed each time you run an action on it. However, you may also *persist* an RDD in memory using the `persist` (or `cache`) method, in which case Spark will keep the elements around on the cluster for much faster access the next time you query it. There is also support for persisting datasets on disk, or replicated across the cluster. The next section in this document describes these options. The following tables list the transformations and actions currently supported (see also the [RDD API doc](api/scala/index.html#org.apache.spark.rdd.RDD) for details): @@ -279,8 +279,8 @@ it is computed in an action, it will be kept in memory on the nodes. The cache i if any partition of an RDD is lost, it will automatically be recomputed using the transformations that originally created it. -In addition, each RDD can be stored using a different *storage level*, allowing you, for example, to -persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), +In addition, each persisted RDD can be stored using a different *storage level*, allowing you, for example, +to persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), or replicate it across nodes, or store the data in off-heap memory in [Tachyon](http://tachyon-project.org/). These levels are chosen by passing a [`org.apache.spark.storage.StorageLevel`](api/scala/index.html#org.apache.spark.storage.StorageLevel) @@ -330,6 +330,8 @@ available storage levels is:
      Master URLMeaning
      local Run Spark locally with one worker thread (i.e. no parallelism at all).
      local[K] Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine). +
      local[K] Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine).
      local[*] Run Spark locally with as many worker threads as logical cores on your machine.
      spark://HOST:PORT Connect to the given Spark standalone cluster master. The port must be whichever one your master is configured to use, which is 7077 by default.
      +Spark sometimes automatically persists intermediate state from RDD operations, even without users calling persist() or cache(). In particular, if a shuffle happens when computing an RDD, Spark will keep the outputs from the map side of the shuffle on disk to avoid re-computing the entire dependency graph if an RDD is re-used. We still recommend users call persist() if they plan to re-use an RDD iteratively. + ### Which Storage Level to Choose? Spark's storage levels are meant to provide different trade-offs between memory usage and CPU From fdae095de2daa1fc3b343c05e515235756d856a4 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Tue, 6 May 2014 21:55:05 -0700 Subject: [PATCH 265/641] [HOTFIX] SPARK-1637: There are some Streaming examples added after the PR #571 was last updated. This resulted in Compilation Errors. cc @mateiz project not compiling currently. Author: Sandeep Closes #673 from techaddict/SPARK-1637-HOTFIX and squashes the following commits: b512f4f [Sandeep] [SPARK-1637][HOTFIX] There are some Streaming examples added after the PR #571 was last updated. This resulted in Compilation Errors. --- .../examples => examples/streaming}/JavaCustomReceiver.java | 6 ++---- .../examples => examples/streaming}/CustomReceiver.scala | 4 ++-- 2 files changed, 4 insertions(+), 6 deletions(-) rename examples/src/main/java/org/apache/spark/{streaming/examples => examples/streaming}/JavaCustomReceiver.java (98%) rename examples/src/main/scala/org/apache/spark/{streaming/examples => examples/streaming}/CustomReceiver.scala (97%) diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java similarity index 98% rename from examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java index e36c7800be23e..7f558f3ee713a 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples; +package org.apache.spark.examples.streaming; import com.google.common.collect.Lists; @@ -48,7 +48,7 @@ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run org.apache.spark.streaming.examples.JavaCustomReceiver local[2] localhost 9999` + * `$ ./run org.apache.spark.examples.streaming.JavaCustomReceiver local[2] localhost 9999` */ public class JavaCustomReceiver extends Receiver { @@ -149,5 +149,3 @@ private void receive() { } } } - - diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala similarity index 97% rename from examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala index eebffd824983f..e317e2d36ae43 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import java.io.{InputStreamReader, BufferedReader, InputStream} import java.net.Socket @@ -37,7 +37,7 @@ import org.apache.spark.streaming.receiver.Receiver * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run org.apache.spark.streaming.examples.CustomReceiver local[2] localhost 9999` + * `$ ./run org.apache.spark.examples.streaming.CustomReceiver local[2] localhost 9999` */ object CustomReceiver { def main(args: Array[String]) { From 913a0a9c0a87e164723ebf9616b883b6329bac71 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 7 May 2014 00:11:05 -0700 Subject: [PATCH 266/641] SPARK-1746: Support setting SPARK_JAVA_OPTS on executors for backwards compatibility Author: Patrick Wendell Closes #676 from pwendell/worker-opts and squashes the following commits: 54456c4 [Patrick Wendell] SPARK-1746: Support setting SPARK_JAVA_OPTS on executors for backwards compatibility --- .../org/apache/spark/deploy/worker/CommandUtils.scala | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 3e615e753b342..c7f0f244ea2fc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -47,9 +47,16 @@ object CommandUtils extends Logging { */ def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") - // Note, this will coalesce multiple options into a single command component val extraOpts = command.extraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq()) + // Exists for backwards compatibility with older Spark versions + val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString) + .getOrElse(Nil) + if (workerLocalOpts.length > 0) { + logWarning("SPARK_JAVA_OPTS was set on the worker. It is deprecated in Spark 1.0.") + logWarning("Set SPARK_LOCAL_DIRS for node-specific storage locations.") + } + val libraryOpts = if (command.libraryPathEntries.size > 0) { val joined = command.libraryPathEntries.mkString(File.pathSeparator) @@ -66,7 +73,7 @@ object CommandUtils extends Logging { val userClassPath = command.classPathEntries ++ Seq(classPath) Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++ - libraryOpts ++ extraOpts ++ memoryOpts + libraryOpts ++ extraOpts ++ workerLocalOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ From 3eb53bd59e828275471d41730e6de601a887416d Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 7 May 2014 03:37:12 -0400 Subject: [PATCH 267/641] [WIP][Spark-SQL] Optimize the Constant Folding for Expression Currently, expression does not support the "constant null" well in constant folding. e.g. Sum(a, 0) actually always produces Literal(0, NumericType) in runtime. For example: ``` explain select isnull(key+null) from src; == Logical Plan == Project [HiveGenericUdf#isnull((key#30 + CAST(null, IntegerType))) AS c_0#28] MetastoreRelation default, src, None == Optimized Logical Plan == Project [true AS c_0#28] MetastoreRelation default, src, None == Physical Plan == Project [true AS c_0#28] HiveTableScan [], (MetastoreRelation default, src, None), None ``` I've create a new Optimization rule called NullPropagation for such kind of constant folding. Author: Cheng Hao Author: Michael Armbrust Closes #482 from chenghao-intel/optimize_constant_folding and squashes the following commits: 2f14b50 [Cheng Hao] Fix code style issues 68b9fad [Cheng Hao] Remove the Literal pattern matching for NullPropagation 29c8166 [Cheng Hao] Update the code for feedback of code review 50444cc [Cheng Hao] Remove the unnecessary null checking 80f9f18 [Cheng Hao] Update the UnitTest for aggregation constant folding 27ea3d7 [Cheng Hao] Fix Constant Folding Bugs & Add More Unittests b28e03a [Cheng Hao] Merge pull request #1 from marmbrus/pr/482 9ccefdb [Michael Armbrust] Add tests for optimized expression evaluation. 543ef9d [Cheng Hao] fix code style issues 9cf0396 [Cheng Hao] update code according to the code review comment 536c005 [Cheng Hao] Add Exceptional case for constant folding 3c045c7 [Cheng Hao] Optimize the Constant Folding by adding more rules 2645d4f [Cheng Hao] Constant Folding(null propagation) --- .../spark/sql/catalyst/dsl/package.scala | 22 +- .../sql/catalyst/expressions/Expression.scala | 1 - .../sql/catalyst/expressions/SortOrder.scala | 6 +- .../catalyst/expressions/complexTypes.scala | 34 +- .../sql/catalyst/expressions/predicates.scala | 3 +- .../sql/catalyst/optimizer/Optimizer.scala | 67 ++++ .../ExpressionEvaluationSuite.scala | 115 ++++++- .../ExpressionOptimizationSuite.scala | 36 ++ .../org/apache/spark/sql/hive/hiveUdfs.scala | 11 + ...M_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0 | 309 ++++++++++++++++++ ...M_COUNT-0-a393cfc24ad74f930f3284743254c10c | 309 ++++++++++++++++++ ...M_COUNT-0-ae497f1556f548c1e2da9244397a985d | 309 ++++++++++++++++++ ...M_COUNT-0-c60b6075da793b826db5eb4b08d7bab9 | 309 ++++++++++++++++++ .../sql/hive/execution/HiveQuerySuite.scala | 3 + 14 files changed, 1502 insertions(+), 32 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala create mode 100644 sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0 create mode 100644 sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-a393cfc24ad74f930f3284743254c10c create mode 100644 sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-ae497f1556f548c1e2da9244397a985d create mode 100644 sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-c60b6075da793b826db5eb4b08d7bab9 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 987befe8e22ee..dc83485df195c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -114,37 +114,37 @@ package object dsl { def attr = analysis.UnresolvedAttribute(s) /** Creates a new AttributeReference of type boolean */ - def boolean = AttributeReference(s, BooleanType, nullable = false)() + def boolean = AttributeReference(s, BooleanType, nullable = true)() /** Creates a new AttributeReference of type byte */ - def byte = AttributeReference(s, ByteType, nullable = false)() + def byte = AttributeReference(s, ByteType, nullable = true)() /** Creates a new AttributeReference of type short */ - def short = AttributeReference(s, ShortType, nullable = false)() + def short = AttributeReference(s, ShortType, nullable = true)() /** Creates a new AttributeReference of type int */ - def int = AttributeReference(s, IntegerType, nullable = false)() + def int = AttributeReference(s, IntegerType, nullable = true)() /** Creates a new AttributeReference of type long */ - def long = AttributeReference(s, LongType, nullable = false)() + def long = AttributeReference(s, LongType, nullable = true)() /** Creates a new AttributeReference of type float */ - def float = AttributeReference(s, FloatType, nullable = false)() + def float = AttributeReference(s, FloatType, nullable = true)() /** Creates a new AttributeReference of type double */ - def double = AttributeReference(s, DoubleType, nullable = false)() + def double = AttributeReference(s, DoubleType, nullable = true)() /** Creates a new AttributeReference of type string */ - def string = AttributeReference(s, StringType, nullable = false)() + def string = AttributeReference(s, StringType, nullable = true)() /** Creates a new AttributeReference of type decimal */ - def decimal = AttributeReference(s, DecimalType, nullable = false)() + def decimal = AttributeReference(s, DecimalType, nullable = true)() /** Creates a new AttributeReference of type timestamp */ - def timestamp = AttributeReference(s, TimestampType, nullable = false)() + def timestamp = AttributeReference(s, TimestampType, nullable = true)() /** Creates a new AttributeReference of type binary */ - def binary = AttributeReference(s, BinaryType, nullable = false)() + def binary = AttributeReference(s, BinaryType, nullable = true)() } implicit class DslAttribute(a: AttributeReference) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index dd9332ada80dd..41398ff956edd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -44,7 +44,6 @@ abstract class Expression extends TreeNode[Expression] { * - A [[expressions.Cast Cast]] or [[expressions.UnaryMinus UnaryMinus]] is foldable if its * child is foldable. */ - // TODO: Supporting more foldable expressions. For example, deterministic Hive UDFs. def foldable: Boolean = false def nullable: Boolean def references: Set[Attribute] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala index 08b2f11d20f5e..d2b7685e73065 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.trees abstract sealed class SortDirection case object Ascending extends SortDirection @@ -27,7 +28,10 @@ case object Descending extends SortDirection * An expression that can be used to sort a tuple. This class extends expression primarily so that * transformations over expression will descend into its child. */ -case class SortOrder(child: Expression, direction: SortDirection) extends UnaryExpression { +case class SortOrder(child: Expression, direction: SortDirection) extends Expression + with trees.UnaryNode[Expression] { + + override def references = child.references override def dataType = child.dataType override def nullable = child.nullable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index c947155cb701c..195ca2eb3d589 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -28,6 +28,7 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { val children = child :: ordinal :: Nil /** `Null` is returned for invalid ordinals. */ override def nullable = true + override def foldable = child.foldable && ordinal.foldable override def references = children.flatMap(_.references).toSet def dataType = child.dataType match { case ArrayType(dt) => dt @@ -40,23 +41,27 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { override def toString = s"$child[$ordinal]" override def eval(input: Row): Any = { - if (child.dataType.isInstanceOf[ArrayType]) { - val baseValue = child.eval(input).asInstanceOf[Seq[_]] - val o = ordinal.eval(input).asInstanceOf[Int] - if (baseValue == null) { - null - } else if (o >= baseValue.size || o < 0) { - null - } else { - baseValue(o) - } + val value = child.eval(input) + if (value == null) { + null } else { - val baseValue = child.eval(input).asInstanceOf[Map[Any, _]] val key = ordinal.eval(input) - if (baseValue == null) { + if (key == null) { null } else { - baseValue.get(key).orNull + if (child.dataType.isInstanceOf[ArrayType]) { + val baseValue = value.asInstanceOf[Seq[_]] + val o = key.asInstanceOf[Int] + if (o >= baseValue.size || o < 0) { + null + } else { + baseValue(o) + } + } else { + val baseValue = value.asInstanceOf[Map[Any, _]] + val key = ordinal.eval(input) + baseValue.get(key).orNull + } } } } @@ -69,7 +74,8 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio type EvaluatedType = Any def dataType = field.dataType - def nullable = field.nullable + override def nullable = field.nullable + override def foldable = child.foldable protected def structType = child.dataType match { case s: StructType => s diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 82c7af684459f..6ee479939d25c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -65,8 +65,7 @@ abstract class BinaryPredicate extends BinaryExpression with Predicate { def nullable = left.nullable || right.nullable } -case class Not(child: Expression) extends Predicate with trees.UnaryNode[Expression] { - def references = child.references +case class Not(child: Expression) extends UnaryExpression with Predicate { override def foldable = child.foldable def nullable = child.nullable override def toString = s"NOT $child" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index c0a09a16ac98d..3037d45cc6e35 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.types._ object Optimizer extends RuleExecutor[LogicalPlan] { val batches = Batch("ConstantFolding", Once, + NullPropagation, ConstantFolding, BooleanSimplification, SimplifyFilters, @@ -85,6 +86,72 @@ object ColumnPruning extends Rule[LogicalPlan] { } } +/** + * Replaces [[catalyst.expressions.Expression Expressions]] that can be statically evaluated with + * equivalent [[catalyst.expressions.Literal Literal]] values. This rule is more specific with + * Null value propagation from bottom to top of the expression tree. + */ +object NullPropagation extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case q: LogicalPlan => q transformExpressionsUp { + case e @ Count(Literal(null, _)) => Literal(0, e.dataType) + case e @ Sum(Literal(c, _)) if c == 0 => Literal(0, e.dataType) + case e @ Average(Literal(c, _)) if c == 0 => Literal(0.0, e.dataType) + case e @ IsNull(c) if c.nullable == false => Literal(false, BooleanType) + case e @ IsNotNull(c) if c.nullable == false => Literal(true, BooleanType) + case e @ GetItem(Literal(null, _), _) => Literal(null, e.dataType) + case e @ GetItem(_, Literal(null, _)) => Literal(null, e.dataType) + case e @ GetField(Literal(null, _), _) => Literal(null, e.dataType) + case e @ Coalesce(children) => { + val newChildren = children.filter(c => c match { + case Literal(null, _) => false + case _ => true + }) + if (newChildren.length == 0) { + Literal(null, e.dataType) + } else if (newChildren.length == 1) { + newChildren(0) + } else { + Coalesce(newChildren) + } + } + case e @ If(Literal(v, _), trueValue, falseValue) => if (v == true) trueValue else falseValue + case e @ In(Literal(v, _), list) if (list.exists(c => c match { + case Literal(candidate, _) if candidate == v => true + case _ => false + })) => Literal(true, BooleanType) + case e: UnaryMinus => e.child match { + case Literal(null, _) => Literal(null, e.dataType) + case _ => e + } + case e: Cast => e.child match { + case Literal(null, _) => Literal(null, e.dataType) + case _ => e + } + case e: Not => e.child match { + case Literal(null, _) => Literal(null, e.dataType) + case _ => e + } + // Put exceptional cases above if any + case e: BinaryArithmetic => e.children match { + case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) + case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) + case _ => e + } + case e: BinaryComparison => e.children match { + case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) + case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) + case _ => e + } + case e: StringRegexExpression => e.children match { + case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) + case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) + case _ => e + } + } + } +} + /** * Replaces [[catalyst.expressions.Expression Expressions]] that can be statically evaluated with * equivalent [[catalyst.expressions.Literal Literal]] values. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index d287ad73b9e9f..91605d0a260e5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -108,9 +108,7 @@ class ExpressionEvaluationSuite extends FunSuite { truthTable.foreach { case (l,r,answer) => val expr = op(Literal(l, BooleanType), Literal(r, BooleanType)) - val result = expr.eval(null) - if (result != answer) - fail(s"$expr should not evaluate to $result, expected: $answer") + checkEvaluation(expr, answer) } } } @@ -131,6 +129,7 @@ class ExpressionEvaluationSuite extends FunSuite { test("LIKE literal Regular Expression") { checkEvaluation(Literal(null, StringType).like("a"), null) + checkEvaluation(Literal("a", StringType).like(Literal(null, StringType)), null) checkEvaluation(Literal(null, StringType).like(Literal(null, StringType)), null) checkEvaluation("abdef" like "abdef", true) checkEvaluation("a_%b" like "a\\__b", true) @@ -159,9 +158,14 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("abc" like regEx, true, new GenericRow(Array[Any]("a%"))) checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("b%"))) checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("bc%"))) + + checkEvaluation(Literal(null, StringType) like regEx, null, new GenericRow(Array[Any]("bc%"))) } test("RLIKE literal Regular Expression") { + checkEvaluation(Literal(null, StringType) rlike "abdef", null) + checkEvaluation("abdef" rlike Literal(null, StringType), null) + checkEvaluation(Literal(null, StringType) rlike Literal(null, StringType), null) checkEvaluation("abdef" rlike "abdef", true) checkEvaluation("abbbbc" rlike "a.*c", true) @@ -257,6 +261,8 @@ class ExpressionEvaluationSuite extends FunSuite { assert(("abcdef" cast DecimalType).nullable === true) assert(("abcdef" cast DoubleType).nullable === true) assert(("abcdef" cast FloatType).nullable === true) + + checkEvaluation(Cast(Literal(null, IntegerType), ShortType), null) } test("timestamp") { @@ -287,5 +293,108 @@ class ExpressionEvaluationSuite extends FunSuite { // A test for higher precision than millis checkEvaluation(Cast(Cast(0.00000001, TimestampType), DoubleType), 0.00000001) } + + test("null checking") { + val row = new GenericRow(Array[Any]("^Ba*n", null, true, null)) + val c1 = 'a.string.at(0) + val c2 = 'a.string.at(1) + val c3 = 'a.boolean.at(2) + val c4 = 'a.boolean.at(3) + + checkEvaluation(IsNull(c1), false, row) + checkEvaluation(IsNotNull(c1), true, row) + + checkEvaluation(IsNull(c2), true, row) + checkEvaluation(IsNotNull(c2), false, row) + + checkEvaluation(IsNull(Literal(1, ShortType)), false) + checkEvaluation(IsNotNull(Literal(1, ShortType)), true) + + checkEvaluation(IsNull(Literal(null, ShortType)), true) + checkEvaluation(IsNotNull(Literal(null, ShortType)), false) + + checkEvaluation(Coalesce(c1 :: c2 :: Nil), "^Ba*n", row) + checkEvaluation(Coalesce(Literal(null, StringType) :: Nil), null, row) + checkEvaluation(Coalesce(Literal(null, StringType) :: c1 :: c2 :: Nil), "^Ba*n", row) + + checkEvaluation(If(c3, Literal("a", StringType), Literal("b", StringType)), "a", row) + checkEvaluation(If(c3, c1, c2), "^Ba*n", row) + checkEvaluation(If(c4, c2, c1), "^Ba*n", row) + checkEvaluation(If(Literal(null, BooleanType), c2, c1), "^Ba*n", row) + checkEvaluation(If(Literal(true, BooleanType), c1, c2), "^Ba*n", row) + checkEvaluation(If(Literal(false, BooleanType), c2, c1), "^Ba*n", row) + checkEvaluation(If(Literal(false, BooleanType), + Literal("a", StringType), Literal("b", StringType)), "b", row) + + checkEvaluation(In(c1, c1 :: c2 :: Nil), true, row) + checkEvaluation(In(Literal("^Ba*n", StringType), + Literal("^Ba*n", StringType) :: Nil), true, row) + checkEvaluation(In(Literal("^Ba*n", StringType), + Literal("^Ba*n", StringType) :: c2 :: Nil), true, row) + } + + test("complex type") { + val row = new GenericRow(Array[Any]( + "^Ba*n", // 0 + null.asInstanceOf[String], // 1 + new GenericRow(Array[Any]("aa", "bb")), // 2 + Map("aa"->"bb"), // 3 + Seq("aa", "bb") // 4 + )) + + val typeS = StructType( + StructField("a", StringType, true) :: StructField("b", StringType, true) :: Nil + ) + val typeMap = MapType(StringType, StringType) + val typeArray = ArrayType(StringType) + + checkEvaluation(GetItem(BoundReference(3, AttributeReference("c", typeMap)()), + Literal("aa")), "bb", row) + checkEvaluation(GetItem(Literal(null, typeMap), Literal("aa")), null, row) + checkEvaluation(GetItem(Literal(null, typeMap), Literal(null, StringType)), null, row) + checkEvaluation(GetItem(BoundReference(3, AttributeReference("c", typeMap)()), + Literal(null, StringType)), null, row) + + checkEvaluation(GetItem(BoundReference(4, AttributeReference("c", typeArray)()), + Literal(1)), "bb", row) + checkEvaluation(GetItem(Literal(null, typeArray), Literal(1)), null, row) + checkEvaluation(GetItem(Literal(null, typeArray), Literal(null, IntegerType)), null, row) + checkEvaluation(GetItem(BoundReference(4, AttributeReference("c", typeArray)()), + Literal(null, IntegerType)), null, row) + + checkEvaluation(GetField(BoundReference(2, AttributeReference("c", typeS)()), "a"), "aa", row) + checkEvaluation(GetField(Literal(null, typeS), "a"), null, row) + } + + test("arithmetic") { + val row = new GenericRow(Array[Any](1, 2, 3, null)) + val c1 = 'a.int.at(0) + val c2 = 'a.int.at(1) + val c3 = 'a.int.at(2) + val c4 = 'a.int.at(3) + + checkEvaluation(UnaryMinus(c1), -1, row) + checkEvaluation(UnaryMinus(Literal(100, IntegerType)), -100) + + checkEvaluation(Add(c1, c4), null, row) + checkEvaluation(Add(c1, c2), 3, row) + checkEvaluation(Add(c1, Literal(null, IntegerType)), null, row) + checkEvaluation(Add(Literal(null, IntegerType), c2), null, row) + checkEvaluation(Add(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + } + + test("BinaryComparison") { + val row = new GenericRow(Array[Any](1, 2, 3, null)) + val c1 = 'a.int.at(0) + val c2 = 'a.int.at(1) + val c3 = 'a.int.at(2) + val c4 = 'a.int.at(3) + + checkEvaluation(LessThan(c1, c4), null, row) + checkEvaluation(LessThan(c1, c2), true, row) + checkEvaluation(LessThan(c1, Literal(null, IntegerType)), null, row) + checkEvaluation(LessThan(Literal(null, IntegerType), c2), null, row) + checkEvaluation(LessThan(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala new file mode 100644 index 0000000000000..890d6289b9dfb --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ + +/** + * Overrides our expression evaluation tests and reruns them after optimization has occured. This + * is to ensure that constant folding and other optimizations do not break anything. + */ +class ExpressionOptimizationSuite extends ExpressionEvaluationSuite { + override def checkEvaluation( + expression: Expression, + expected: Any, + inputRow: Row = EmptyRow): Unit = { + val plan = Project(Alias(expression, s"Optimized($expression)")() :: Nil, NoRelation) + val optimizedPlan = Optimizer(plan) + super.checkEvaluation(optimizedPlan.expressions.head, expected, inputRow) + } +} \ No newline at end of file diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index c7de4ab6d3955..d50e2c65b7b36 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -22,6 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.ql.exec.UDF import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} +import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic._ import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.primitive._ @@ -237,6 +238,16 @@ private[hive] case class HiveGenericUdf(name: String, children: Seq[Expression]) @transient protected lazy val returnInspector = function.initialize(argumentInspectors.toArray) + @transient + protected lazy val isUDFDeterministic = { + val udfType = function.getClass().getAnnotation(classOf[HiveUDFType]) + (udfType != null && udfType.deterministic()) + } + + override def foldable = { + isUDFDeterministic && children.foldLeft(true)((prev, n) => prev && n.foldable) + } + val dataType: DataType = inspectorToDataType(returnInspector) override def eval(input: Row): Any = { diff --git a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0 b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0 new file mode 100644 index 0000000000000..7643569a2c234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0 @@ -0,0 +1,309 @@ +0.0 0 0 0 3 +2.0 0 2 0 1 +4.0 0 4 0 1 +5.0 0 15 0 3 +8.0 0 8 0 1 +9.0 0 9 0 1 +10.0 0 10 0 1 +11.0 0 11 0 1 +12.0 0 24 0 2 +15.0 0 30 0 2 +17.0 0 17 0 1 +18.0 0 36 0 2 +19.0 0 19 0 1 +20.0 0 20 0 1 +24.0 0 48 0 2 +26.0 0 52 0 2 +27.0 0 27 0 1 +28.0 0 28 0 1 +30.0 0 30 0 1 +33.0 0 33 0 1 +34.0 0 34 0 1 +35.0 0 105 0 3 +37.0 0 74 0 2 +41.0 0 41 0 1 +42.0 0 84 0 2 +43.0 0 43 0 1 +44.0 0 44 0 1 +47.0 0 47 0 1 +51.0 0 102 0 2 +53.0 0 53 0 1 +54.0 0 54 0 1 +57.0 0 57 0 1 +58.0 0 116 0 2 +64.0 0 64 0 1 +65.0 0 65 0 1 +66.0 0 66 0 1 +67.0 0 134 0 2 +69.0 0 69 0 1 +70.0 0 210 0 3 +72.0 0 144 0 2 +74.0 0 74 0 1 +76.0 0 152 0 2 +77.0 0 77 0 1 +78.0 0 78 0 1 +80.0 0 80 0 1 +82.0 0 82 0 1 +83.0 0 166 0 2 +84.0 0 168 0 2 +85.0 0 85 0 1 +86.0 0 86 0 1 +87.0 0 87 0 1 +90.0 0 270 0 3 +92.0 0 92 0 1 +95.0 0 190 0 2 +96.0 0 96 0 1 +97.0 0 194 0 2 +98.0 0 196 0 2 +100.0 0 200 0 2 +103.0 0 206 0 2 +104.0 0 208 0 2 +105.0 0 105 0 1 +111.0 0 111 0 1 +113.0 0 226 0 2 +114.0 0 114 0 1 +116.0 0 116 0 1 +118.0 0 236 0 2 +119.0 0 357 0 3 +120.0 0 240 0 2 +125.0 0 250 0 2 +126.0 0 126 0 1 +128.0 0 384 0 3 +129.0 0 258 0 2 +131.0 0 131 0 1 +133.0 0 133 0 1 +134.0 0 268 0 2 +136.0 0 136 0 1 +137.0 0 274 0 2 +138.0 0 552 0 4 +143.0 0 143 0 1 +145.0 0 145 0 1 +146.0 0 292 0 2 +149.0 0 298 0 2 +150.0 0 150 0 1 +152.0 0 304 0 2 +153.0 0 153 0 1 +155.0 0 155 0 1 +156.0 0 156 0 1 +157.0 0 157 0 1 +158.0 0 158 0 1 +160.0 0 160 0 1 +162.0 0 162 0 1 +163.0 0 163 0 1 +164.0 0 328 0 2 +165.0 0 330 0 2 +166.0 0 166 0 1 +167.0 0 501 0 3 +168.0 0 168 0 1 +169.0 0 676 0 4 +170.0 0 170 0 1 +172.0 0 344 0 2 +174.0 0 348 0 2 +175.0 0 350 0 2 +176.0 0 352 0 2 +177.0 0 177 0 1 +178.0 0 178 0 1 +179.0 0 358 0 2 +180.0 0 180 0 1 +181.0 0 181 0 1 +183.0 0 183 0 1 +186.0 0 186 0 1 +187.0 0 561 0 3 +189.0 0 189 0 1 +190.0 0 190 0 1 +191.0 0 382 0 2 +192.0 0 192 0 1 +193.0 0 579 0 3 +194.0 0 194 0 1 +195.0 0 390 0 2 +196.0 0 196 0 1 +197.0 0 394 0 2 +199.0 0 597 0 3 +200.0 0 400 0 2 +201.0 0 201 0 1 +202.0 0 202 0 1 +203.0 0 406 0 2 +205.0 0 410 0 2 +207.0 0 414 0 2 +208.0 0 624 0 3 +209.0 0 418 0 2 +213.0 0 426 0 2 +214.0 0 214 0 1 +216.0 0 432 0 2 +217.0 0 434 0 2 +218.0 0 218 0 1 +219.0 0 438 0 2 +221.0 0 442 0 2 +222.0 0 222 0 1 +223.0 0 446 0 2 +224.0 0 448 0 2 +226.0 0 226 0 1 +228.0 0 228 0 1 +229.0 0 458 0 2 +230.0 0 1150 0 5 +233.0 0 466 0 2 +235.0 0 235 0 1 +237.0 0 474 0 2 +238.0 0 476 0 2 +239.0 0 478 0 2 +241.0 0 241 0 1 +242.0 0 484 0 2 +244.0 0 244 0 1 +247.0 0 247 0 1 +248.0 0 248 0 1 +249.0 0 249 0 1 +252.0 0 252 0 1 +255.0 0 510 0 2 +256.0 0 512 0 2 +257.0 0 257 0 1 +258.0 0 258 0 1 +260.0 0 260 0 1 +262.0 0 262 0 1 +263.0 0 263 0 1 +265.0 0 530 0 2 +266.0 0 266 0 1 +272.0 0 544 0 2 +273.0 0 819 0 3 +274.0 0 274 0 1 +275.0 0 275 0 1 +277.0 0 1108 0 4 +278.0 0 556 0 2 +280.0 0 560 0 2 +281.0 0 562 0 2 +282.0 0 564 0 2 +283.0 0 283 0 1 +284.0 0 284 0 1 +285.0 0 285 0 1 +286.0 0 286 0 1 +287.0 0 287 0 1 +288.0 0 576 0 2 +289.0 0 289 0 1 +291.0 0 291 0 1 +292.0 0 292 0 1 +296.0 0 296 0 1 +298.0 0 894 0 3 +302.0 0 302 0 1 +305.0 0 305 0 1 +306.0 0 306 0 1 +307.0 0 614 0 2 +308.0 0 308 0 1 +309.0 0 618 0 2 +310.0 0 310 0 1 +311.0 0 933 0 3 +315.0 0 315 0 1 +316.0 0 948 0 3 +317.0 0 634 0 2 +318.0 0 954 0 3 +321.0 0 642 0 2 +322.0 0 644 0 2 +323.0 0 323 0 1 +325.0 0 650 0 2 +327.0 0 981 0 3 +331.0 0 662 0 2 +332.0 0 332 0 1 +333.0 0 666 0 2 +335.0 0 335 0 1 +336.0 0 336 0 1 +338.0 0 338 0 1 +339.0 0 339 0 1 +341.0 0 341 0 1 +342.0 0 684 0 2 +344.0 0 688 0 2 +345.0 0 345 0 1 +348.0 0 1740 0 5 +351.0 0 351 0 1 +353.0 0 706 0 2 +356.0 0 356 0 1 +360.0 0 360 0 1 +362.0 0 362 0 1 +364.0 0 364 0 1 +365.0 0 365 0 1 +366.0 0 366 0 1 +367.0 0 734 0 2 +368.0 0 368 0 1 +369.0 0 1107 0 3 +373.0 0 373 0 1 +374.0 0 374 0 1 +375.0 0 375 0 1 +377.0 0 377 0 1 +378.0 0 378 0 1 +379.0 0 379 0 1 +382.0 0 764 0 2 +384.0 0 1152 0 3 +386.0 0 386 0 1 +389.0 0 389 0 1 +392.0 0 392 0 1 +393.0 0 393 0 1 +394.0 0 394 0 1 +395.0 0 790 0 2 +396.0 0 1188 0 3 +397.0 0 794 0 2 +399.0 0 798 0 2 +400.0 0 400 0 1 +401.0 0 2005 0 5 +402.0 0 402 0 1 +403.0 0 1209 0 3 +404.0 0 808 0 2 +406.0 0 1624 0 4 +407.0 0 407 0 1 +409.0 0 1227 0 3 +411.0 0 411 0 1 +413.0 0 826 0 2 +414.0 0 828 0 2 +417.0 0 1251 0 3 +418.0 0 418 0 1 +419.0 0 419 0 1 +421.0 0 421 0 1 +424.0 0 848 0 2 +427.0 0 427 0 1 +429.0 0 858 0 2 +430.0 0 1290 0 3 +431.0 0 1293 0 3 +432.0 0 432 0 1 +435.0 0 435 0 1 +436.0 0 436 0 1 +437.0 0 437 0 1 +438.0 0 1314 0 3 +439.0 0 878 0 2 +443.0 0 443 0 1 +444.0 0 444 0 1 +446.0 0 446 0 1 +448.0 0 448 0 1 +449.0 0 449 0 1 +452.0 0 452 0 1 +453.0 0 453 0 1 +454.0 0 1362 0 3 +455.0 0 455 0 1 +457.0 0 457 0 1 +458.0 0 916 0 2 +459.0 0 918 0 2 +460.0 0 460 0 1 +462.0 0 924 0 2 +463.0 0 926 0 2 +466.0 0 1398 0 3 +467.0 0 467 0 1 +468.0 0 1872 0 4 +469.0 0 2345 0 5 +470.0 0 470 0 1 +472.0 0 472 0 1 +475.0 0 475 0 1 +477.0 0 477 0 1 +478.0 0 956 0 2 +479.0 0 479 0 1 +480.0 0 1440 0 3 +481.0 0 481 0 1 +482.0 0 482 0 1 +483.0 0 483 0 1 +484.0 0 484 0 1 +485.0 0 485 0 1 +487.0 0 487 0 1 +489.0 0 1956 0 4 +490.0 0 490 0 1 +491.0 0 491 0 1 +492.0 0 984 0 2 +493.0 0 493 0 1 +494.0 0 494 0 1 +495.0 0 495 0 1 +496.0 0 496 0 1 +497.0 0 497 0 1 +498.0 0 1494 0 3 diff --git a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-a393cfc24ad74f930f3284743254c10c b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-a393cfc24ad74f930f3284743254c10c new file mode 100644 index 0000000000000..f23b45c32ecca --- /dev/null +++ b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-a393cfc24ad74f930f3284743254c10c @@ -0,0 +1,309 @@ +0.0 0.0 0 0 0 3 +0.0 2.0 0 2 0 1 +0.0 4.0 0 4 0 1 +0.0 5.0 0 15 0 3 +0.0 8.0 0 8 0 1 +0.0 9.0 0 9 0 1 +0.0 10.0 0 10 0 1 +0.0 11.0 0 11 0 1 +0.0 12.0 0 24 0 2 +0.0 15.0 0 30 0 2 +0.0 17.0 0 17 0 1 +0.0 18.0 0 36 0 2 +0.0 19.0 0 19 0 1 +0.0 20.0 0 20 0 1 +0.0 24.0 0 48 0 2 +0.0 26.0 0 52 0 2 +0.0 27.0 0 27 0 1 +0.0 28.0 0 28 0 1 +0.0 30.0 0 30 0 1 +0.0 33.0 0 33 0 1 +0.0 34.0 0 34 0 1 +0.0 35.0 0 105 0 3 +0.0 37.0 0 74 0 2 +0.0 41.0 0 41 0 1 +0.0 42.0 0 84 0 2 +0.0 43.0 0 43 0 1 +0.0 44.0 0 44 0 1 +0.0 47.0 0 47 0 1 +0.0 51.0 0 102 0 2 +0.0 53.0 0 53 0 1 +0.0 54.0 0 54 0 1 +0.0 57.0 0 57 0 1 +0.0 58.0 0 116 0 2 +0.0 64.0 0 64 0 1 +0.0 65.0 0 65 0 1 +0.0 66.0 0 66 0 1 +0.0 67.0 0 134 0 2 +0.0 69.0 0 69 0 1 +0.0 70.0 0 210 0 3 +0.0 72.0 0 144 0 2 +0.0 74.0 0 74 0 1 +0.0 76.0 0 152 0 2 +0.0 77.0 0 77 0 1 +0.0 78.0 0 78 0 1 +0.0 80.0 0 80 0 1 +0.0 82.0 0 82 0 1 +0.0 83.0 0 166 0 2 +0.0 84.0 0 168 0 2 +0.0 85.0 0 85 0 1 +0.0 86.0 0 86 0 1 +0.0 87.0 0 87 0 1 +0.0 90.0 0 270 0 3 +0.0 92.0 0 92 0 1 +0.0 95.0 0 190 0 2 +0.0 96.0 0 96 0 1 +0.0 97.0 0 194 0 2 +0.0 98.0 0 196 0 2 +0.0 100.0 0 200 0 2 +0.0 103.0 0 206 0 2 +0.0 104.0 0 208 0 2 +0.0 105.0 0 105 0 1 +0.0 111.0 0 111 0 1 +0.0 113.0 0 226 0 2 +0.0 114.0 0 114 0 1 +0.0 116.0 0 116 0 1 +0.0 118.0 0 236 0 2 +0.0 119.0 0 357 0 3 +0.0 120.0 0 240 0 2 +0.0 125.0 0 250 0 2 +0.0 126.0 0 126 0 1 +0.0 128.0 0 384 0 3 +0.0 129.0 0 258 0 2 +0.0 131.0 0 131 0 1 +0.0 133.0 0 133 0 1 +0.0 134.0 0 268 0 2 +0.0 136.0 0 136 0 1 +0.0 137.0 0 274 0 2 +0.0 138.0 0 552 0 4 +0.0 143.0 0 143 0 1 +0.0 145.0 0 145 0 1 +0.0 146.0 0 292 0 2 +0.0 149.0 0 298 0 2 +0.0 150.0 0 150 0 1 +0.0 152.0 0 304 0 2 +0.0 153.0 0 153 0 1 +0.0 155.0 0 155 0 1 +0.0 156.0 0 156 0 1 +0.0 157.0 0 157 0 1 +0.0 158.0 0 158 0 1 +0.0 160.0 0 160 0 1 +0.0 162.0 0 162 0 1 +0.0 163.0 0 163 0 1 +0.0 164.0 0 328 0 2 +0.0 165.0 0 330 0 2 +0.0 166.0 0 166 0 1 +0.0 167.0 0 501 0 3 +0.0 168.0 0 168 0 1 +0.0 169.0 0 676 0 4 +0.0 170.0 0 170 0 1 +0.0 172.0 0 344 0 2 +0.0 174.0 0 348 0 2 +0.0 175.0 0 350 0 2 +0.0 176.0 0 352 0 2 +0.0 177.0 0 177 0 1 +0.0 178.0 0 178 0 1 +0.0 179.0 0 358 0 2 +0.0 180.0 0 180 0 1 +0.0 181.0 0 181 0 1 +0.0 183.0 0 183 0 1 +0.0 186.0 0 186 0 1 +0.0 187.0 0 561 0 3 +0.0 189.0 0 189 0 1 +0.0 190.0 0 190 0 1 +0.0 191.0 0 382 0 2 +0.0 192.0 0 192 0 1 +0.0 193.0 0 579 0 3 +0.0 194.0 0 194 0 1 +0.0 195.0 0 390 0 2 +0.0 196.0 0 196 0 1 +0.0 197.0 0 394 0 2 +0.0 199.0 0 597 0 3 +0.0 200.0 0 400 0 2 +0.0 201.0 0 201 0 1 +0.0 202.0 0 202 0 1 +0.0 203.0 0 406 0 2 +0.0 205.0 0 410 0 2 +0.0 207.0 0 414 0 2 +0.0 208.0 0 624 0 3 +0.0 209.0 0 418 0 2 +0.0 213.0 0 426 0 2 +0.0 214.0 0 214 0 1 +0.0 216.0 0 432 0 2 +0.0 217.0 0 434 0 2 +0.0 218.0 0 218 0 1 +0.0 219.0 0 438 0 2 +0.0 221.0 0 442 0 2 +0.0 222.0 0 222 0 1 +0.0 223.0 0 446 0 2 +0.0 224.0 0 448 0 2 +0.0 226.0 0 226 0 1 +0.0 228.0 0 228 0 1 +0.0 229.0 0 458 0 2 +0.0 230.0 0 1150 0 5 +0.0 233.0 0 466 0 2 +0.0 235.0 0 235 0 1 +0.0 237.0 0 474 0 2 +0.0 238.0 0 476 0 2 +0.0 239.0 0 478 0 2 +0.0 241.0 0 241 0 1 +0.0 242.0 0 484 0 2 +0.0 244.0 0 244 0 1 +0.0 247.0 0 247 0 1 +0.0 248.0 0 248 0 1 +0.0 249.0 0 249 0 1 +0.0 252.0 0 252 0 1 +0.0 255.0 0 510 0 2 +0.0 256.0 0 512 0 2 +0.0 257.0 0 257 0 1 +0.0 258.0 0 258 0 1 +0.0 260.0 0 260 0 1 +0.0 262.0 0 262 0 1 +0.0 263.0 0 263 0 1 +0.0 265.0 0 530 0 2 +0.0 266.0 0 266 0 1 +0.0 272.0 0 544 0 2 +0.0 273.0 0 819 0 3 +0.0 274.0 0 274 0 1 +0.0 275.0 0 275 0 1 +0.0 277.0 0 1108 0 4 +0.0 278.0 0 556 0 2 +0.0 280.0 0 560 0 2 +0.0 281.0 0 562 0 2 +0.0 282.0 0 564 0 2 +0.0 283.0 0 283 0 1 +0.0 284.0 0 284 0 1 +0.0 285.0 0 285 0 1 +0.0 286.0 0 286 0 1 +0.0 287.0 0 287 0 1 +0.0 288.0 0 576 0 2 +0.0 289.0 0 289 0 1 +0.0 291.0 0 291 0 1 +0.0 292.0 0 292 0 1 +0.0 296.0 0 296 0 1 +0.0 298.0 0 894 0 3 +0.0 302.0 0 302 0 1 +0.0 305.0 0 305 0 1 +0.0 306.0 0 306 0 1 +0.0 307.0 0 614 0 2 +0.0 308.0 0 308 0 1 +0.0 309.0 0 618 0 2 +0.0 310.0 0 310 0 1 +0.0 311.0 0 933 0 3 +0.0 315.0 0 315 0 1 +0.0 316.0 0 948 0 3 +0.0 317.0 0 634 0 2 +0.0 318.0 0 954 0 3 +0.0 321.0 0 642 0 2 +0.0 322.0 0 644 0 2 +0.0 323.0 0 323 0 1 +0.0 325.0 0 650 0 2 +0.0 327.0 0 981 0 3 +0.0 331.0 0 662 0 2 +0.0 332.0 0 332 0 1 +0.0 333.0 0 666 0 2 +0.0 335.0 0 335 0 1 +0.0 336.0 0 336 0 1 +0.0 338.0 0 338 0 1 +0.0 339.0 0 339 0 1 +0.0 341.0 0 341 0 1 +0.0 342.0 0 684 0 2 +0.0 344.0 0 688 0 2 +0.0 345.0 0 345 0 1 +0.0 348.0 0 1740 0 5 +0.0 351.0 0 351 0 1 +0.0 353.0 0 706 0 2 +0.0 356.0 0 356 0 1 +0.0 360.0 0 360 0 1 +0.0 362.0 0 362 0 1 +0.0 364.0 0 364 0 1 +0.0 365.0 0 365 0 1 +0.0 366.0 0 366 0 1 +0.0 367.0 0 734 0 2 +0.0 368.0 0 368 0 1 +0.0 369.0 0 1107 0 3 +0.0 373.0 0 373 0 1 +0.0 374.0 0 374 0 1 +0.0 375.0 0 375 0 1 +0.0 377.0 0 377 0 1 +0.0 378.0 0 378 0 1 +0.0 379.0 0 379 0 1 +0.0 382.0 0 764 0 2 +0.0 384.0 0 1152 0 3 +0.0 386.0 0 386 0 1 +0.0 389.0 0 389 0 1 +0.0 392.0 0 392 0 1 +0.0 393.0 0 393 0 1 +0.0 394.0 0 394 0 1 +0.0 395.0 0 790 0 2 +0.0 396.0 0 1188 0 3 +0.0 397.0 0 794 0 2 +0.0 399.0 0 798 0 2 +0.0 400.0 0 400 0 1 +0.0 401.0 0 2005 0 5 +0.0 402.0 0 402 0 1 +0.0 403.0 0 1209 0 3 +0.0 404.0 0 808 0 2 +0.0 406.0 0 1624 0 4 +0.0 407.0 0 407 0 1 +0.0 409.0 0 1227 0 3 +0.0 411.0 0 411 0 1 +0.0 413.0 0 826 0 2 +0.0 414.0 0 828 0 2 +0.0 417.0 0 1251 0 3 +0.0 418.0 0 418 0 1 +0.0 419.0 0 419 0 1 +0.0 421.0 0 421 0 1 +0.0 424.0 0 848 0 2 +0.0 427.0 0 427 0 1 +0.0 429.0 0 858 0 2 +0.0 430.0 0 1290 0 3 +0.0 431.0 0 1293 0 3 +0.0 432.0 0 432 0 1 +0.0 435.0 0 435 0 1 +0.0 436.0 0 436 0 1 +0.0 437.0 0 437 0 1 +0.0 438.0 0 1314 0 3 +0.0 439.0 0 878 0 2 +0.0 443.0 0 443 0 1 +0.0 444.0 0 444 0 1 +0.0 446.0 0 446 0 1 +0.0 448.0 0 448 0 1 +0.0 449.0 0 449 0 1 +0.0 452.0 0 452 0 1 +0.0 453.0 0 453 0 1 +0.0 454.0 0 1362 0 3 +0.0 455.0 0 455 0 1 +0.0 457.0 0 457 0 1 +0.0 458.0 0 916 0 2 +0.0 459.0 0 918 0 2 +0.0 460.0 0 460 0 1 +0.0 462.0 0 924 0 2 +0.0 463.0 0 926 0 2 +0.0 466.0 0 1398 0 3 +0.0 467.0 0 467 0 1 +0.0 468.0 0 1872 0 4 +0.0 469.0 0 2345 0 5 +0.0 470.0 0 470 0 1 +0.0 472.0 0 472 0 1 +0.0 475.0 0 475 0 1 +0.0 477.0 0 477 0 1 +0.0 478.0 0 956 0 2 +0.0 479.0 0 479 0 1 +0.0 480.0 0 1440 0 3 +0.0 481.0 0 481 0 1 +0.0 482.0 0 482 0 1 +0.0 483.0 0 483 0 1 +0.0 484.0 0 484 0 1 +0.0 485.0 0 485 0 1 +0.0 487.0 0 487 0 1 +0.0 489.0 0 1956 0 4 +0.0 490.0 0 490 0 1 +0.0 491.0 0 491 0 1 +0.0 492.0 0 984 0 2 +0.0 493.0 0 493 0 1 +0.0 494.0 0 494 0 1 +0.0 495.0 0 495 0 1 +0.0 496.0 0 496 0 1 +0.0 497.0 0 497 0 1 +0.0 498.0 0 1494 0 3 diff --git a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-ae497f1556f548c1e2da9244397a985d b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-ae497f1556f548c1e2da9244397a985d new file mode 100644 index 0000000000000..7839d714c25d2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-ae497f1556f548c1e2da9244397a985d @@ -0,0 +1,309 @@ +0 3 +0 1 +0 1 +0 3 +0 1 +0 1 +0 1 +0 1 +0 2 +0 2 +0 1 +0 2 +0 1 +0 1 +0 2 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 3 +0 2 +0 1 +0 2 +0 1 +0 1 +0 1 +0 2 +0 1 +0 1 +0 1 +0 2 +0 1 +0 1 +0 1 +0 2 +0 1 +0 3 +0 2 +0 1 +0 2 +0 1 +0 1 +0 1 +0 1 +0 2 +0 2 +0 1 +0 1 +0 1 +0 3 +0 1 +0 2 +0 1 +0 2 +0 2 +0 2 +0 2 +0 2 +0 1 +0 1 +0 2 +0 1 +0 1 +0 2 +0 3 +0 2 +0 2 +0 1 +0 3 +0 2 +0 1 +0 1 +0 2 +0 1 +0 2 +0 4 +0 1 +0 1 +0 2 +0 2 +0 1 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 2 +0 2 +0 1 +0 3 +0 1 +0 4 +0 1 +0 2 +0 2 +0 2 +0 2 +0 1 +0 1 +0 2 +0 1 +0 1 +0 1 +0 1 +0 3 +0 1 +0 1 +0 2 +0 1 +0 3 +0 1 +0 2 +0 1 +0 2 +0 3 +0 2 +0 1 +0 1 +0 2 +0 2 +0 2 +0 3 +0 2 +0 2 +0 1 +0 2 +0 2 +0 1 +0 2 +0 2 +0 1 +0 2 +0 2 +0 1 +0 1 +0 2 +0 5 +0 2 +0 1 +0 2 +0 2 +0 2 +0 1 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 2 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 2 +0 1 +0 2 +0 3 +0 1 +0 1 +0 4 +0 2 +0 2 +0 2 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 2 +0 1 +0 1 +0 1 +0 1 +0 3 +0 1 +0 1 +0 1 +0 2 +0 1 +0 2 +0 1 +0 3 +0 1 +0 3 +0 2 +0 3 +0 2 +0 2 +0 1 +0 2 +0 3 +0 2 +0 1 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 2 +0 2 +0 1 +0 5 +0 1 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 2 +0 1 +0 3 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 2 +0 3 +0 1 +0 1 +0 1 +0 1 +0 1 +0 2 +0 3 +0 2 +0 2 +0 1 +0 5 +0 1 +0 3 +0 2 +0 4 +0 1 +0 3 +0 1 +0 2 +0 2 +0 3 +0 1 +0 1 +0 1 +0 2 +0 1 +0 2 +0 3 +0 3 +0 1 +0 1 +0 1 +0 1 +0 3 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 3 +0 1 +0 1 +0 2 +0 2 +0 1 +0 2 +0 2 +0 3 +0 1 +0 4 +0 5 +0 1 +0 1 +0 1 +0 1 +0 2 +0 1 +0 3 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 4 +0 1 +0 1 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 3 diff --git a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-c60b6075da793b826db5eb4b08d7bab9 b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-c60b6075da793b826db5eb4b08d7bab9 new file mode 100644 index 0000000000000..cd6b14ac2a425 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-c60b6075da793b826db5eb4b08d7bab9 @@ -0,0 +1,309 @@ +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 4 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 4 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 5 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 4 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 5 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 5 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 4 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 4 +0.0 0 0 5 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 4 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 3 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index d224d2ee600ec..87a92d83383ab 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -33,6 +33,9 @@ class HiveQuerySuite extends HiveComparisonTest { hiveql("FROM src SELECT key").collect() } + createQueryTest("Constant Folding Optimization for AVG_SUM_COUNT", + "SELECT AVG(0), SUM(0), COUNT(null), COUNT(value) FROM src GROUP BY key") + createQueryTest("Simple Average", "SELECT AVG(key) FROM src") From 967635a2425a769b932eea0984fe697d6721cab0 Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Wed, 7 May 2014 09:41:31 -0700 Subject: [PATCH 268/641] [SPARK-1460] Returning SchemaRDD instead of normal RDD on Set operations... ... that do not change schema Author: Kan Zhang Closes #448 from kanzhang/SPARK-1460 and squashes the following commits: 111e388 [Kan Zhang] silence MiMa errors in EdgeRDD and VertexRDD 91dc787 [Kan Zhang] Taking into account newly added Ordering param 79ed52a [Kan Zhang] [SPARK-1460] Returning SchemaRDD on Set operations that do not change schema --- .../main/scala/org/apache/spark/rdd/RDD.scala | 10 +- .../org/apache/spark/graphx/EdgeRDD.scala | 10 +- .../org/apache/spark/graphx/VertexRDD.scala | 10 +- project/MimaBuild.scala | 2 + python/pyspark/sql.py | 29 ++++ .../org/apache/spark/sql/SchemaRDD.scala | 67 ++++++++- .../spark/sql/api/java/JavaSchemaRDD.scala | 140 ++++++++++++++++++ 7 files changed, 246 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 3b3524f33e811..a1ca612cc9a09 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -128,7 +128,7 @@ abstract class RDD[T: ClassTag]( @transient var name: String = null /** Assign a name to this RDD */ - def setName(_name: String): RDD[T] = { + def setName(_name: String): this.type = { name = _name this } @@ -138,7 +138,7 @@ abstract class RDD[T: ClassTag]( * it is computed. This can only be used to assign a new storage level if the RDD does not * have a storage level set yet.. */ - def persist(newLevel: StorageLevel): RDD[T] = { + def persist(newLevel: StorageLevel): this.type = { // TODO: Handle changes of StorageLevel if (storageLevel != StorageLevel.NONE && newLevel != storageLevel) { throw new UnsupportedOperationException( @@ -152,10 +152,10 @@ abstract class RDD[T: ClassTag]( } /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - def persist(): RDD[T] = persist(StorageLevel.MEMORY_ONLY) + def persist(): this.type = persist(StorageLevel.MEMORY_ONLY) /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - def cache(): RDD[T] = persist() + def cache(): this.type = persist() /** * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. @@ -163,7 +163,7 @@ abstract class RDD[T: ClassTag]( * @param blocking Whether to block until all blocks are deleted. * @return This RDD. */ - def unpersist(blocking: Boolean = true): RDD[T] = { + def unpersist(blocking: Boolean = true): this.type = { logInfo("Removing RDD " + id + " from persistence list") sc.unpersistRDD(id, blocking) storageLevel = StorageLevel.NONE diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index 6d04bf790e3a5..fa78ca99b8891 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -51,18 +51,12 @@ class EdgeRDD[@specialized ED: ClassTag]( override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect() - override def persist(newLevel: StorageLevel): EdgeRDD[ED] = { + override def persist(newLevel: StorageLevel): this.type = { partitionsRDD.persist(newLevel) this } - /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - override def persist(): EdgeRDD[ED] = persist(StorageLevel.MEMORY_ONLY) - - /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - override def cache(): EdgeRDD[ED] = persist() - - override def unpersist(blocking: Boolean = true): EdgeRDD[ED] = { + override def unpersist(blocking: Boolean = true): this.type = { partitionsRDD.unpersist(blocking) this } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index d6788d4d4b9fd..f0fc605c88575 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -71,18 +71,12 @@ class VertexRDD[@specialized VD: ClassTag]( override protected def getPreferredLocations(s: Partition): Seq[String] = partitionsRDD.preferredLocations(s) - override def persist(newLevel: StorageLevel): VertexRDD[VD] = { + override def persist(newLevel: StorageLevel): this.type = { partitionsRDD.persist(newLevel) this } - /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - override def persist(): VertexRDD[VD] = persist(StorageLevel.MEMORY_ONLY) - - /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - override def cache(): VertexRDD[VD] = persist() - - override def unpersist(blocking: Boolean = true): VertexRDD[VD] = { + override def unpersist(blocking: Boolean = true): this.type = { partitionsRDD.unpersist(blocking) this } diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index d540dc0a986e9..efdb38e907d14 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -74,6 +74,8 @@ object MimaBuild { ) ++ excludeSparkClass("rdd.ClassTags") ++ excludeSparkClass("util.XORShiftRandom") ++ + excludeSparkClass("graphx.EdgeRDD") ++ + excludeSparkClass("graphx.VertexRDD") ++ excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ excludeSparkClass("mllib.optimization.SquaredGradient") ++ excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 1a62031db5c41..6789d7002b3b7 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -360,6 +360,35 @@ def getCheckpointFile(self): else: return None + def coalesce(self, numPartitions, shuffle=False): + rdd = self._jschema_rdd.coalesce(numPartitions, shuffle) + return SchemaRDD(rdd, self.sql_ctx) + + def distinct(self): + rdd = self._jschema_rdd.distinct() + return SchemaRDD(rdd, self.sql_ctx) + + def intersection(self, other): + if (other.__class__ is SchemaRDD): + rdd = self._jschema_rdd.intersection(other._jschema_rdd) + return SchemaRDD(rdd, self.sql_ctx) + else: + raise ValueError("Can only intersect with another SchemaRDD") + + def repartition(self, numPartitions): + rdd = self._jschema_rdd.repartition(numPartitions) + return SchemaRDD(rdd, self.sql_ctx) + + def subtract(self, other, numPartitions=None): + if (other.__class__ is SchemaRDD): + if numPartitions is None: + rdd = self._jschema_rdd.subtract(other._jschema_rdd) + else: + rdd = self._jschema_rdd.subtract(other._jschema_rdd, numPartitions) + return SchemaRDD(rdd, self.sql_ctx) + else: + raise ValueError("Can only subtract another SchemaRDD") + def _test(): import doctest from pyspark.context import SparkContext diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index d7782d6b32819..34200be3ac955 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -19,14 +19,16 @@ package org.apache.spark.sql import net.razorvine.pickle.Pickler -import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} +import org.apache.spark.{Dependency, OneToOneDependency, Partition, Partitioner, TaskContext} import org.apache.spark.annotation.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.api.java.JavaSchemaRDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.types.BooleanType +import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} import org.apache.spark.api.java.JavaRDD import java.util.{Map => JMap} @@ -296,6 +298,13 @@ class SchemaRDD( */ def toSchemaRDD = this + /** + * Returns this RDD as a JavaSchemaRDD. + * + * @group schema + */ + def toJavaSchemaRDD: JavaSchemaRDD = new JavaSchemaRDD(sqlContext, logicalPlan) + private[sql] def javaToPython: JavaRDD[Array[Byte]] = { val fieldNames: Seq[String] = this.queryExecution.analyzed.output.map(_.name) this.mapPartitions { iter => @@ -314,4 +323,60 @@ class SchemaRDD( } } } + + /** + * Creates SchemaRDD by applying own schema to derived RDD. Typically used to wrap return value + * of base RDD functions that do not change schema. + * + * @param rdd RDD derived from this one and has same schema + * + * @group schema + */ + private def applySchema(rdd: RDD[Row]): SchemaRDD = { + new SchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(logicalPlan.output, rdd))) + } + + // ======================================================================= + // Base RDD functions that do NOT change schema + // ======================================================================= + + // Transformations (return a new RDD) + + override def coalesce(numPartitions: Int, shuffle: Boolean = false) + (implicit ord: Ordering[Row] = null): SchemaRDD = + applySchema(super.coalesce(numPartitions, shuffle)(ord)) + + override def distinct(): SchemaRDD = + applySchema(super.distinct()) + + override def distinct(numPartitions: Int) + (implicit ord: Ordering[Row] = null): SchemaRDD = + applySchema(super.distinct(numPartitions)(ord)) + + override def filter(f: Row => Boolean): SchemaRDD = + applySchema(super.filter(f)) + + override def intersection(other: RDD[Row]): SchemaRDD = + applySchema(super.intersection(other)) + + override def intersection(other: RDD[Row], partitioner: Partitioner) + (implicit ord: Ordering[Row] = null): SchemaRDD = + applySchema(super.intersection(other, partitioner)(ord)) + + override def intersection(other: RDD[Row], numPartitions: Int): SchemaRDD = + applySchema(super.intersection(other, numPartitions)) + + override def repartition(numPartitions: Int) + (implicit ord: Ordering[Row] = null): SchemaRDD = + applySchema(super.repartition(numPartitions)(ord)) + + override def subtract(other: RDD[Row]): SchemaRDD = + applySchema(super.subtract(other)) + + override def subtract(other: RDD[Row], numPartitions: Int): SchemaRDD = + applySchema(super.subtract(other, numPartitions)) + + override def subtract(other: RDD[Row], p: Partitioner) + (implicit ord: Ordering[Row] = null): SchemaRDD = + applySchema(super.subtract(other, p)(ord)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala index d43d672938f51..22f57b758dd02 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala @@ -17,10 +17,13 @@ package org.apache.spark.sql.api.java +import org.apache.spark.Partitioner import org.apache.spark.api.java.{JavaRDDLike, JavaRDD} +import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.sql.{SQLContext, SchemaRDD, SchemaRDDLike} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel /** * An RDD of [[Row]] objects that is returned as the result of a Spark SQL query. In addition to @@ -45,4 +48,141 @@ class JavaSchemaRDD( override def wrapRDD(rdd: RDD[Row]): JavaRDD[Row] = JavaRDD.fromRDD(rdd) val rdd = baseSchemaRDD.map(new Row(_)) + + override def toString: String = baseSchemaRDD.toString + + // ======================================================================= + // Base RDD functions that do NOT change schema + // ======================================================================= + + // Common RDD functions + + /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ + def cache(): JavaSchemaRDD = { + baseSchemaRDD.cache() + this + } + + /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ + def persist(): JavaSchemaRDD = { + baseSchemaRDD.persist() + this + } + + /** + * Set this RDD's storage level to persist its values across operations after the first time + * it is computed. This can only be used to assign a new storage level if the RDD does not + * have a storage level set yet.. + */ + def persist(newLevel: StorageLevel): JavaSchemaRDD = { + baseSchemaRDD.persist(newLevel) + this + } + + /** + * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. + * + * @param blocking Whether to block until all blocks are deleted. + * @return This RDD. + */ + def unpersist(blocking: Boolean = true): JavaSchemaRDD = { + baseSchemaRDD.unpersist(blocking) + this + } + + /** Assign a name to this RDD */ + def setName(name: String): JavaSchemaRDD = { + baseSchemaRDD.setName(name) + this + } + + // Transformations (return a new RDD) + + /** + * Return a new RDD that is reduced into `numPartitions` partitions. + */ + def coalesce(numPartitions: Int, shuffle: Boolean = false): JavaSchemaRDD = + baseSchemaRDD.coalesce(numPartitions, shuffle).toJavaSchemaRDD + + /** + * Return a new RDD containing the distinct elements in this RDD. + */ + def distinct(): JavaSchemaRDD = + baseSchemaRDD.distinct().toJavaSchemaRDD + + /** + * Return a new RDD containing the distinct elements in this RDD. + */ + def distinct(numPartitions: Int): JavaSchemaRDD = + baseSchemaRDD.distinct(numPartitions).toJavaSchemaRDD + + /** + * Return a new RDD containing only the elements that satisfy a predicate. + */ + def filter(f: JFunction[Row, java.lang.Boolean]): JavaSchemaRDD = + baseSchemaRDD.filter(x => f.call(new Row(x)).booleanValue()).toJavaSchemaRDD + + /** + * Return the intersection of this RDD and another one. The output will not contain any + * duplicate elements, even if the input RDDs did. + * + * Note that this method performs a shuffle internally. + */ + def intersection(other: JavaSchemaRDD): JavaSchemaRDD = + this.baseSchemaRDD.intersection(other.baseSchemaRDD).toJavaSchemaRDD + + /** + * Return the intersection of this RDD and another one. The output will not contain any + * duplicate elements, even if the input RDDs did. + * + * Note that this method performs a shuffle internally. + * + * @param partitioner Partitioner to use for the resulting RDD + */ + def intersection(other: JavaSchemaRDD, partitioner: Partitioner): JavaSchemaRDD = + this.baseSchemaRDD.intersection(other.baseSchemaRDD, partitioner).toJavaSchemaRDD + + /** + * Return the intersection of this RDD and another one. The output will not contain any + * duplicate elements, even if the input RDDs did. Performs a hash partition across the cluster + * + * Note that this method performs a shuffle internally. + * + * @param numPartitions How many partitions to use in the resulting RDD + */ + def intersection(other: JavaSchemaRDD, numPartitions: Int): JavaSchemaRDD = + this.baseSchemaRDD.intersection(other.baseSchemaRDD, numPartitions).toJavaSchemaRDD + + /** + * Return a new RDD that has exactly `numPartitions` partitions. + * + * Can increase or decrease the level of parallelism in this RDD. Internally, this uses + * a shuffle to redistribute data. + * + * If you are decreasing the number of partitions in this RDD, consider using `coalesce`, + * which can avoid performing a shuffle. + */ + def repartition(numPartitions: Int): JavaSchemaRDD = + baseSchemaRDD.repartition(numPartitions).toJavaSchemaRDD + + /** + * Return an RDD with the elements from `this` that are not in `other`. + * + * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting + * RDD will be <= us. + */ + def subtract(other: JavaSchemaRDD): JavaSchemaRDD = + this.baseSchemaRDD.subtract(other.baseSchemaRDD).toJavaSchemaRDD + + /** + * Return an RDD with the elements from `this` that are not in `other`. + */ + def subtract(other: JavaSchemaRDD, numPartitions: Int): JavaSchemaRDD = + this.baseSchemaRDD.subtract(other.baseSchemaRDD, numPartitions).toJavaSchemaRDD + + /** + * Return an RDD with the elements from `this` that are not in `other`. + */ + def subtract(other: JavaSchemaRDD, p: Partitioner): JavaSchemaRDD = + this.baseSchemaRDD.subtract(other.baseSchemaRDD, p).toJavaSchemaRDD } From 3308722ca03f2bfa792e9a2cff9c894b967983d9 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Wed, 7 May 2014 09:48:31 -0700 Subject: [PATCH 269/641] SPARK-1579: Clean up PythonRDD and avoid swallowing IOExceptions This patch includes several cleanups to PythonRDD, focused around fixing [SPARK-1579](https://issues.apache.org/jira/browse/SPARK-1579) cleanly. Listed in order of approximate importance: - The Python daemon waits for Spark to close the socket before exiting, in order to avoid causing spurious IOExceptions in Spark's `PythonRDD::WriterThread`. - Removes the Python Monitor Thread, which polled for task cancellations in order to kill the Python worker. Instead, we do this in the onCompleteCallback, since this is guaranteed to be called during cancellation. - Adds a "completed" variable to TaskContext to avoid the issue noted in [SPARK-1019](https://issues.apache.org/jira/browse/SPARK-1019), where onCompleteCallbacks may be execution-order dependent. Along with this, I removed the "context.interrupted = true" flag in the onCompleteCallback. - Extracts PythonRDD::WriterThread to its own class. Since this patch provides an alternative solution to [SPARK-1019](https://issues.apache.org/jira/browse/SPARK-1019), I did test it with ``` sc.textFile("latlon.tsv").take(5) ``` many times without error. Additionally, in order to test the unswallowed exceptions, I performed ``` sc.textFile("s3n://").count() ``` and cut my internet during execution. Prior to this patch, we got the "stdin writer exited early" message, which was unhelpful. Now, we get the SocketExceptions propagated through Spark to the user and get proper (though unsuccessful) task retries. Author: Aaron Davidson Closes #640 from aarondav/pyspark-io and squashes the following commits: b391ff8 [Aaron Davidson] Detect "clean socket shutdowns" and stop waiting on the socket c0c49da [Aaron Davidson] SPARK-1579: Clean up PythonRDD and avoid swallowing IOExceptions --- .../scala/org/apache/spark/TaskContext.scala | 5 + .../apache/spark/api/python/PythonRDD.scala | 217 +++++++++--------- .../spark/scheduler/ShuffleMapTask.scala | 10 +- python/pyspark/context.py | 2 +- python/pyspark/daemon.py | 14 +- 5 files changed, 141 insertions(+), 107 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index dc012cc381346..fc4812753d005 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -42,9 +42,13 @@ class TaskContext( // List of callback functions to execute when the task completes. @transient private val onCompleteCallbacks = new ArrayBuffer[() => Unit] + // Set to true when the task is completed, before the onCompleteCallbacks are executed. + @volatile var completed: Boolean = false + /** * Add a callback function to be executed on task completion. An example use * is for HadoopRDD to register a callback to close the input stream. + * Will be called in any situation - success, failure, or cancellation. * @param f Callback function. */ def addOnCompleteCallback(f: () => Unit) { @@ -52,6 +56,7 @@ class TaskContext( } def executeOnCompleteCallbacks() { + completed = true // 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 61407007087c6..fecd9762f3f60 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 @@ -56,122 +56,37 @@ private[spark] class PythonRDD[T: ClassTag]( val env = SparkEnv.get val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap) - // Ensure worker socket is closed on task completion. Closing sockets is idempotent. - context.addOnCompleteCallback(() => + // Start a thread to feed the process input from our parent's iterator + val writerThread = new WriterThread(env, worker, split, context) + + context.addOnCompleteCallback { () => + writerThread.shutdownOnTaskCompletion() + + // Cleanup the worker socket. This will also cause the Python worker to exit. try { worker.close() } catch { case e: Exception => logWarning("Failed to close worker socket", e) } - ) - - @volatile var readerException: Exception = null - - // Start a thread to feed the process input from our parent's iterator - new Thread("stdin writer for " + pythonExec) { - override def run() { - try { - SparkEnv.set(env) - val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) - val dataOut = new DataOutputStream(stream) - // Partition index - dataOut.writeInt(split.index) - // sparkFilesDir - PythonRDD.writeUTF(SparkFiles.getRootDirectory, dataOut) - // Broadcast variables - dataOut.writeInt(broadcastVars.length) - for (broadcast <- broadcastVars) { - dataOut.writeLong(broadcast.id) - dataOut.writeInt(broadcast.value.length) - dataOut.write(broadcast.value) - } - // Python includes (*.zip and *.egg files) - dataOut.writeInt(pythonIncludes.length) - for (include <- pythonIncludes) { - PythonRDD.writeUTF(include, dataOut) - } - dataOut.flush() - // Serialized command: - dataOut.writeInt(command.length) - dataOut.write(command) - // Data values - PythonRDD.writeIteratorToStream(parent.iterator(split, context), dataOut) - dataOut.flush() - worker.shutdownOutput() - } catch { - - case e: java.io.FileNotFoundException => - readerException = e - Try(worker.shutdownOutput()) // kill Python worker process - - case e: IOException => - // This can happen for legitimate reasons if the Python code stops returning data - // before we are done passing elements through, e.g., for take(). Just log a message to - // say it happened (as it could also be hiding a real IOException from a data source). - logInfo("stdin writer to Python finished early (may not be an error)", e) - - case e: Exception => - // We must avoid throwing exceptions here, because the thread uncaught exception handler - // will kill the whole executor (see Executor). - readerException = e - Try(worker.shutdownOutput()) // kill Python worker process - } - } - }.start() - - // Necessary to distinguish between a task that has failed and a task that is finished - @volatile var complete: Boolean = false - - // It is necessary to have a monitor thread for python workers if the user cancels with - // interrupts disabled. In that case we will need to explicitly kill the worker, otherwise the - // threads can block indefinitely. - new Thread(s"Worker Monitor for $pythonExec") { - override def run() { - // Kill the worker if it is interrupted or completed - // When a python task completes, the context is always set to interupted - while (!context.interrupted) { - Thread.sleep(2000) - } - if (!complete) { - try { - logWarning("Incomplete task interrupted: Attempting to kill Python Worker") - env.destroyPythonWorker(pythonExec, envVars.toMap) - } catch { - case e: Exception => - logError("Exception when trying to kill worker", e) - } - } - } - }.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{ () => - complete = true // Indicate that the task has completed successfully - context.interrupted = true } + writerThread.start() + new MonitorThread(env, worker, context).start() + // 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]] { def next(): Array[Byte] = { val obj = _nextObj if (hasNext) { - // FIXME: can deadlock if worker is waiting for us to - // respond to current message (currently irrelevant because - // output is shutdown before we read any input) _nextObj = read() } obj } private def read(): Array[Byte] = { - if (readerException != null) { - throw readerException + if (writerThread.exception.isDefined) { + throw writerThread.exception.get } try { stream.readInt() match { @@ -190,13 +105,14 @@ private[spark] class PythonRDD[T: ClassTag]( val total = finishTime - startTime logInfo("Times: total = %s, boot = %s, init = %s, finish = %s".format(total, boot, init, finish)) - read + read() case SpecialLengths.PYTHON_EXCEPTION_THROWN => // Signals that an exception has been thrown in python val exLength = stream.readInt() val obj = new Array[Byte](exLength) stream.readFully(obj) - throw new PythonException(new String(obj, "utf-8"), readerException) + throw new PythonException(new String(obj, "utf-8"), + writerThread.exception.getOrElse(null)) case SpecialLengths.END_OF_DATA_SECTION => // We've finished the data section of the output, but we can still // read some accumulator updates: @@ -210,10 +126,15 @@ private[spark] class PythonRDD[T: ClassTag]( Array.empty[Byte] } } catch { - case e: Exception if readerException != null => + + case e: Exception if context.interrupted => + logDebug("Exception thrown after task interruption", e) + throw new TaskKilledException + + case e: Exception if writerThread.exception.isDefined => logError("Python worker exited unexpectedly (crashed)", e) - logError("Python crash may have been caused by prior exception:", readerException) - throw readerException + logError("This may have been caused by a prior exception:", writerThread.exception.get) + throw writerThread.exception.get case eof: EOFException => throw new SparkException("Python worker exited unexpectedly (crashed)", eof) @@ -224,10 +145,100 @@ private[spark] class PythonRDD[T: ClassTag]( def hasNext = _nextObj.length != 0 } - stdoutIterator + new InterruptibleIterator(context, stdoutIterator) } val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this) + + /** + * The thread responsible for writing the data from the PythonRDD's parent iterator to the + * Python process. + */ + class WriterThread(env: SparkEnv, worker: Socket, split: Partition, context: TaskContext) + extends Thread(s"stdout writer for $pythonExec") { + + @volatile private var _exception: Exception = null + + setDaemon(true) + + /** Contains the exception thrown while writing the parent iterator to the Python process. */ + def exception: Option[Exception] = Option(_exception) + + /** Terminates the writer thread, ignoring any exceptions that may occur due to cleanup. */ + def shutdownOnTaskCompletion() { + assert(context.completed) + this.interrupt() + } + + override def run() { + try { + SparkEnv.set(env) + val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) + val dataOut = new DataOutputStream(stream) + // Partition index + dataOut.writeInt(split.index) + // sparkFilesDir + PythonRDD.writeUTF(SparkFiles.getRootDirectory, dataOut) + // Broadcast variables + dataOut.writeInt(broadcastVars.length) + for (broadcast <- broadcastVars) { + dataOut.writeLong(broadcast.id) + dataOut.writeInt(broadcast.value.length) + dataOut.write(broadcast.value) + } + // Python includes (*.zip and *.egg files) + dataOut.writeInt(pythonIncludes.length) + for (include <- pythonIncludes) { + PythonRDD.writeUTF(include, dataOut) + } + dataOut.flush() + // Serialized command: + dataOut.writeInt(command.length) + dataOut.write(command) + // Data values + PythonRDD.writeIteratorToStream(parent.iterator(split, context), dataOut) + dataOut.flush() + } catch { + case e: Exception if context.completed || context.interrupted => + logDebug("Exception thrown after task completion (likely due to cleanup)", e) + + case e: Exception => + // We must avoid throwing exceptions here, because the thread uncaught exception handler + // will kill the whole executor (see org.apache.spark.executor.Executor). + _exception = e + } finally { + Try(worker.shutdownOutput()) // kill Python worker process + } + } + } + + /** + * It is necessary to have a monitor thread for python workers if the user cancels with + * interrupts disabled. In that case we will need to explicitly kill the worker, otherwise the + * threads can block indefinitely. + */ + class MonitorThread(env: SparkEnv, worker: Socket, context: TaskContext) + extends Thread(s"Worker Monitor for $pythonExec") { + + setDaemon(true) + + override def run() { + // Kill the worker if it is interrupted, checking until task completion. + // TODO: This has a race condition if interruption occurs, as completed may still become true. + while (!context.interrupted && !context.completed) { + Thread.sleep(2000) + } + if (!context.completed) { + try { + logWarning("Incomplete task interrupted: Attempting to kill Python Worker") + env.destroyPythonWorker(pythonExec, envVars.toMap) + } catch { + case e: Exception => + logError("Exception when trying to kill worker", e) + } + } + } + } } /** Thrown for exceptions in user Python code. */ 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 02b62de7e36b6..2259df0b56bad 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -17,11 +17,13 @@ package org.apache.spark.scheduler +import scala.language.existentials + import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashMap -import scala.language.existentials +import scala.util.Try import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics @@ -196,7 +198,11 @@ private[spark] class ShuffleMapTask( } finally { // Release the writers back to the shuffle block manager. if (shuffle != null && shuffle.writers != null) { - shuffle.releaseWriters(success) + try { + shuffle.releaseWriters(success) + } catch { + case e: Exception => logError("Failed to release shuffle writers", e) + } } // Execute the callbacks on task completion. context.executeOnCompleteCallbacks() diff --git a/python/pyspark/context.py b/python/pyspark/context.py index c7dc85ea03544..cac133d0fcf6c 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -453,7 +453,7 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): >>> lock = threading.Lock() >>> def map_func(x): ... sleep(100) - ... return x * x + ... raise Exception("Task should have been cancelled") >>> def start_job(x): ... global result ... try: diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index eb18ec08c9139..b2f226a55ec13 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -74,6 +74,17 @@ def handle_sigchld(*args): raise signal.signal(SIGCHLD, handle_sigchld) + # Blocks until the socket is closed by draining the input stream + # until it raises an exception or returns EOF. + def waitSocketClose(sock): + try: + while True: + # Empty string is returned upon EOF (and only then). + if sock.recv(4096) == '': + return + except: + pass + # Handle clients while not should_exit(): # Wait until a client arrives or we have to exit @@ -105,7 +116,8 @@ def handle_sigchld(*args): exit_code = exc.code finally: outfile.flush() - sock.close() + # The Scala side will close the socket upon task completion. + waitSocketClose(sock) os._exit(compute_real_exit_code(exit_code)) else: sock.close() From ca43186867f0120c29d1b27cfee0c7ff4a107d84 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 7 May 2014 16:54:58 -0400 Subject: [PATCH 270/641] [SQL] Fix Performance Issue in data type casting Using lazy val object instead of function in the class Cast, which improved the performance nearly by 2X in my local micro-benchmark. Author: Cheng Hao Closes #679 from chenghao-intel/fix_type_casting and squashes the following commits: 71b0902 [Cheng Hao] using lazy val object instead of function for data type casting --- .../scala/org/apache/spark/sql/catalyst/expressions/Cast.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 40d2b42a0cda3..0b3a4e728ec54 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -182,7 +182,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toFloat(b) } - def cast: Any => Any = dataType match { + private lazy val cast: Any => Any = dataType match { case StringType => castToString case BinaryType => castToBinary case DecimalType => castToDecimal From 7f6f4a1035ae0c9fa2029fe991f621ca263d53e0 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Wed, 7 May 2014 17:24:12 -0400 Subject: [PATCH 271/641] Nicer logging for SecurityManager startup Happy to open a jira ticket if you'd like to track one there. Author: Andrew Ash Closes #678 from ash211/SecurityManagerLogging and squashes the following commits: 2aa0b7a [Andrew Ash] Nicer logging for SecurityManager startup --- core/src/main/scala/org/apache/spark/SecurityManager.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index b4b0067801259..74aa441619bd2 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -146,8 +146,9 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) private val secretKey = generateSecretKey() - logInfo("SecurityManager, is authentication enabled: " + authOn + - " are ui acls enabled: " + uiAclsOn + " users with view permissions: " + viewAcls.toString()) + logInfo("SecurityManager: authentication " + (if (authOn) "enabled" else "disabled") + + "; ui acls " + (if (uiAclsOn) "enabled" else "disabled") + + "; users with view permissions: " + viewAcls.toString()) // Set our own authenticator to properly negotiate user/password for HTTP connections. // This is needed by the HTTP client fetching from the HttpServer. Put here so its From d00981a95185229fd1594d5c030a00f219fb1a14 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Wed, 7 May 2014 17:24:49 -0400 Subject: [PATCH 272/641] Typo fix: fetchting -> fetching Author: Andrew Ash Closes #680 from ash211/patch-3 and squashes the following commits: 9ce3746 [Andrew Ash] Typo fix: fetchting -> fetching --- make-distribution.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/make-distribution.sh b/make-distribution.sh index ebcd8c74fc5a6..759e555b4b69a 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -189,7 +189,7 @@ if [ "$SPARK_TACHYON" == "true" ]; then TMPD=`mktemp -d 2>/dev/null || mktemp -d -t 'disttmp'` pushd $TMPD > /dev/null - echo "Fetchting tachyon tgz" + echo "Fetching tachyon tgz" wget "$TACHYON_URL" tar xf "tachyon-${TACHYON_VERSION}-bin.tar.gz" From 5200872243aa5906dc8a06772e61d75f19557aac Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 7 May 2014 14:35:22 -0700 Subject: [PATCH 273/641] [SPARK-1688] Propagate PySpark worker stderr to driver When at least one of the following conditions is true, PySpark cannot be loaded: 1. PYTHONPATH is not set 2. PYTHONPATH does not contain the python directory (or jar, in the case of YARN) 3. The jar does not contain pyspark files (YARN) 4. The jar does not contain py4j files (YARN) However, we currently throw the same random `java.io.EOFException` for all of the above cases, when trying to read from the python daemon's output. This message is super unhelpful. This PR includes the python stderr and the PYTHONPATH in the exception propagated to the driver. Now, the exception message looks something like: ``` Error from python worker: : No module named pyspark PYTHONPATH was: /path/to/spark/python:/path/to/some/jar java.io.EOFException ``` whereas before it was just ``` java.io.EOFException ``` Author: Andrew Or Closes #603 from andrewor14/pyspark-exception and squashes the following commits: 10d65d3 [Andrew Or] Throwable -> Exception, worker -> daemon 862d1d7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-exception a5ed798 [Andrew Or] Use block string and interpolation instead of var (minor) cc09c45 [Andrew Or] Account for the fact that the python daemon may not have terminated yet 444f019 [Andrew Or] Use the new RedirectThread + include system PYTHONPATH aab00ae [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-exception 0cc2402 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-exception 783efe2 [Andrew Or] Make python daemon stderr indentation consistent 9524172 [Andrew Or] Avoid potential NPE / error stream contention + Move things around 29f9688 [Andrew Or] Add back original exception type e92d36b [Andrew Or] Include python worker stderr in the exception propagated to the driver 7c69360 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-exception cdbc185 [Andrew Or] Fix python attribute not found exception when PYTHONPATH is not set dcc0353 [Andrew Or] Check both python and system environment variables for PYTHONPATH 6c09c21 [Andrew Or] Validate PYTHONPATH and PySpark modules before starting python workers --- .../apache/spark/api/python/PythonUtils.scala | 27 +++- .../api/python/PythonWorkerFactory.scala | 136 ++++++++---------- .../apache/spark/deploy/PythonRunner.scala | 24 +--- .../scala/org/apache/spark/util/Utils.scala | 37 +++++ 4 files changed, 123 insertions(+), 101 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index cf69fa1d53fde..6d3e257c4d5df 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.api.python -import java.io.File +import java.io.{File, InputStream, IOException, OutputStream} import scala.collection.mutable.ArrayBuffer @@ -40,3 +40,28 @@ private[spark] object PythonUtils { paths.filter(_ != "").mkString(File.pathSeparator) } } + + +/** + * A utility class to redirect the child process's stdout or stderr. + */ +private[spark] class RedirectThread( + in: InputStream, + out: OutputStream, + name: String) + extends Thread(name) { + + setDaemon(true) + override def run() { + scala.util.control.Exception.ignoring(classOf[IOException]) { + // FIXME: We copy the stream on the level of bytes to avoid encoding problems. + val buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + out.write(buf, 0, len) + out.flush() + len = in.read(buf) + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index b0bf4e052b3e9..002f2acd94dee 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -17,15 +17,18 @@ package org.apache.spark.api.python -import java.io.{DataInputStream, File, IOException, OutputStreamWriter} +import java.io.{DataInputStream, InputStream, OutputStreamWriter} import java.net.{InetAddress, ServerSocket, Socket, SocketException} import scala.collection.JavaConversions._ import org.apache.spark._ +import org.apache.spark.util.Utils private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String, String]) - extends Logging { + extends Logging { + + import PythonWorkerFactory._ // Because forking processes from Java is expensive, we prefer to launch a single Python daemon // (pyspark/daemon.py) and tell it to fork new workers for our tasks. This daemon currently @@ -38,7 +41,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String var daemonPort: Int = 0 val pythonPath = PythonUtils.mergePythonPaths( - PythonUtils.sparkPythonPath, envVars.getOrElse("PYTHONPATH", "")) + PythonUtils.sparkPythonPath, + envVars.getOrElse("PYTHONPATH", ""), + sys.env.getOrElse("PYTHONPATH", "")) def create(): Socket = { if (useDaemon) { @@ -61,12 +66,11 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String try { new Socket(daemonHost, daemonPort) } catch { - case exc: SocketException => { + case exc: SocketException => logWarning("Python daemon unexpectedly quit, attempting to restart") stopDaemon() startDaemon() new Socket(daemonHost, daemonPort) - } case e: Throwable => throw e } } @@ -87,39 +91,8 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String workerEnv.put("PYTHONPATH", pythonPath) val worker = pb.start() - // Redirect the worker's stderr to ours - new Thread("stderr reader for " + pythonExec) { - setDaemon(true) - override def run() { - scala.util.control.Exception.ignoring(classOf[IOException]) { - // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val in = worker.getErrorStream - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - System.err.write(buf, 0, len) - len = in.read(buf) - } - } - } - }.start() - - // Redirect worker's stdout to our stderr - new Thread("stdout reader for " + pythonExec) { - setDaemon(true) - override def run() { - scala.util.control.Exception.ignoring(classOf[IOException]) { - // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val in = worker.getInputStream - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - System.err.write(buf, 0, len) - len = in.read(buf) - } - } - } - }.start() + // Redirect worker stdout and stderr + redirectStreamsToStderr(worker.getInputStream, worker.getErrorStream) // Tell the worker our port val out = new OutputStreamWriter(worker.getOutputStream) @@ -142,10 +115,6 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String null } - def stop() { - stopDaemon() - } - private def startDaemon() { synchronized { // Is it already running? @@ -161,46 +130,38 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String workerEnv.put("PYTHONPATH", pythonPath) daemon = pb.start() - // Redirect the stderr to ours - new Thread("stderr reader for " + pythonExec) { - setDaemon(true) - override def run() { - scala.util.control.Exception.ignoring(classOf[IOException]) { - // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val in = daemon.getErrorStream - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - System.err.write(buf, 0, len) - len = in.read(buf) - } - } - } - }.start() - val in = new DataInputStream(daemon.getInputStream) daemonPort = in.readInt() - // Redirect further stdout output to our stderr - new Thread("stdout reader for " + pythonExec) { - setDaemon(true) - override def run() { - scala.util.control.Exception.ignoring(classOf[IOException]) { - // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - System.err.write(buf, 0, len) - len = in.read(buf) - } - } - } - }.start() + // Redirect daemon stdout and stderr + redirectStreamsToStderr(in, daemon.getErrorStream) + } catch { - case e: Throwable => { + case e: Exception => + + // If the daemon exists, wait for it to finish and get its stderr + val stderr = Option(daemon) + .flatMap { d => Utils.getStderr(d, PROCESS_WAIT_TIMEOUT_MS) } + .getOrElse("") + stopDaemon() - throw e - } + + if (stderr != "") { + val formattedStderr = stderr.replace("\n", "\n ") + val errorMessage = s""" + |Error from python worker: + | $formattedStderr + |PYTHONPATH was: + | $pythonPath + |$e""" + + // Append error message from python daemon, but keep original stack trace + val wrappedException = new SparkException(errorMessage.stripMargin) + wrappedException.setStackTrace(e.getStackTrace) + throw wrappedException + } else { + throw e + } } // Important: don't close daemon's stdin (daemon.getOutputStream) so it can correctly @@ -208,6 +169,19 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } } + /** + * Redirect the given streams to our stderr in separate threads. + */ + private def redirectStreamsToStderr(stdout: InputStream, stderr: InputStream) { + try { + new RedirectThread(stdout, System.err, "stdout reader for " + pythonExec).start() + new RedirectThread(stderr, System.err, "stderr reader for " + pythonExec).start() + } catch { + case e: Exception => + logError("Exception in redirecting streams", e) + } + } + private def stopDaemon() { synchronized { // Request shutdown of existing daemon by sending SIGTERM @@ -219,4 +193,12 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String daemonPort = 0 } } + + def stop() { + stopDaemon() + } +} + +private object PythonWorkerFactory { + val PROCESS_WAIT_TIMEOUT_MS = 10000 } diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala index f2e7c7a508b3f..e20d4486c8f0c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -17,13 +17,10 @@ package org.apache.spark.deploy -import java.io.{IOException, File, InputStream, OutputStream} - import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ -import org.apache.spark.SparkContext -import org.apache.spark.api.python.PythonUtils +import org.apache.spark.api.python.{PythonUtils, RedirectThread} /** * A main class used by spark-submit to launch Python applications. It executes python as a @@ -62,23 +59,4 @@ object PythonRunner { System.exit(process.waitFor()) } - - /** - * A utility class to redirect the child process's stdout or stderr - */ - class RedirectThread(in: InputStream, out: OutputStream, name: String) extends Thread(name) { - setDaemon(true) - override def run() { - scala.util.control.Exception.ignoring(classOf[IOException]) { - // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - out.write(buf, 0, len) - out.flush() - len = in.read(buf) - } - } - } - } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 202bd46956f87..3f0ed61c5bbfb 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1088,4 +1088,41 @@ private[spark] object Utils extends Logging { def stripDirectory(path: String): String = { path.split(File.separator).last } + + /** + * Wait for a process to terminate for at most the specified duration. + * Return whether the process actually terminated after the given timeout. + */ + def waitForProcess(process: Process, timeoutMs: Long): Boolean = { + var terminated = false + val startTime = System.currentTimeMillis + while (!terminated) { + try { + process.exitValue + terminated = true + } catch { + case e: IllegalThreadStateException => + // Process not terminated yet + if (System.currentTimeMillis - startTime > timeoutMs) { + return false + } + Thread.sleep(100) + } + } + true + } + + /** + * Return the stderr of a process after waiting for the process to terminate. + * If the process does not terminate within the specified timeout, return None. + */ + def getStderr(process: Process, timeoutMs: Long): Option[String] = { + val terminated = Utils.waitForProcess(process, timeoutMs) + if (terminated) { + Some(Source.fromInputStream(process.getErrorStream).getLines().mkString("\n")) + } else { + None + } + } + } From 4bec84b6a23e1e642708a70a6c7ef7b3d1df9b3e Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 7 May 2014 15:51:53 -0700 Subject: [PATCH 274/641] SPARK-1569 Spark on Yarn, authentication broken by pr299 Pass the configs as java options since the executor needs to know before it registers whether to create the connection using authentication or not. We could see about passing only the authentication configs but for now I just had it pass them all. I also updating it to use a list to construct the command to make it the same as ClientBase and avoid any issues with spaces. Author: Thomas Graves Closes #649 from tgravescs/SPARK-1569 and squashes the following commits: 0178ab8 [Thomas Graves] add akka settings 22a8735 [Thomas Graves] Change to only path spark.auth* configs 8ccc1d4 [Thomas Graves] SPARK-1569 Spark on Yarn, authentication broken --- .../deploy/yarn/ExecutorRunnableUtil.scala | 49 ++++++++++++------- 1 file changed, 30 insertions(+), 19 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 96f8aa93394f5..32f8861dc9503 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -21,7 +21,7 @@ import java.io.File import java.net.URI import scala.collection.JavaConversions._ -import scala.collection.mutable.HashMap +import scala.collection.mutable.{HashMap, ListBuffer} import org.apache.hadoop.fs.Path import org.apache.hadoop.yarn.api._ @@ -44,9 +44,9 @@ trait ExecutorRunnableUtil extends Logging { hostname: String, executorMemory: Int, executorCores: Int, - localResources: HashMap[String, LocalResource]) = { + localResources: HashMap[String, LocalResource]): List[String] = { // Extra options for the JVM - var JAVA_OPTS = "" + val JAVA_OPTS = ListBuffer[String]() // Set the JVM memory val executorMemoryString = executorMemory + "m" JAVA_OPTS += "-Xms" + executorMemoryString + " -Xmx" + executorMemoryString + " " @@ -56,10 +56,21 @@ trait ExecutorRunnableUtil extends Logging { JAVA_OPTS += opts } - JAVA_OPTS += " -Djava.io.tmpdir=" + - new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " " + JAVA_OPTS += "-Djava.io.tmpdir=" + + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) JAVA_OPTS += ClientBase.getLog4jConfiguration(localResources) + // Certain configs need to be passed here because they are needed before the Executor + // registers with the Scheduler and transfers the spark configs. Since the Executor backend + // uses Akka to connect to the scheduler, the akka settings are needed as well as the + // authentication settings. + sparkConf.getAll. + filter { case (k, v) => k.startsWith("spark.auth") || k.startsWith("spark.akka") }. + foreach { case (k, v) => JAVA_OPTS += "-D" + k + "=" + "\\\"" + v + "\\\"" } + + sparkConf.getAkkaConf. + foreach { case (k, v) => JAVA_OPTS += "-D" + k + "=" + "\\\"" + v + "\\\"" } + // Commenting it out for now - so that people can refer to the properties if required. Remove // it once cpuset version is pushed out. // The context is, default gc for server class machines end up using all cores to do gc - hence @@ -85,25 +96,25 @@ trait ExecutorRunnableUtil extends Logging { } */ - val commands = List[String]( - Environment.JAVA_HOME.$() + "/bin/java" + - " -server " + + val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", + "-server", // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. // 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 ? - " -XX:OnOutOfMemoryError='kill %p' " + - JAVA_OPTS + - " org.apache.spark.executor.CoarseGrainedExecutorBackend " + - masterAddress + " " + - slaveId + " " + - hostname + " " + - executorCores + - " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" + - " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") - - commands + "-XX:OnOutOfMemoryError='kill %p'") ++ + JAVA_OPTS ++ + Seq("org.apache.spark.executor.CoarseGrainedExecutorBackend", + masterAddress.toString, + slaveId.toString, + hostname.toString, + executorCores.toString, + "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", + "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") + + // TODO: it would be nicer to just make sure there are no null commands here + commands.map(s => if (s == null) "null" else s).toList } private def setupDistributedCache( From 3188553f73970270717a7fee4a116e29ad4becc9 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 7 May 2014 16:01:11 -0700 Subject: [PATCH 275/641] [SPARK-1743][MLLIB] add loadLibSVMFile and saveAsLibSVMFile to pyspark Make loading/saving labeled data easier for pyspark users. Also changed type check in `SparseVector` to allow numpy integers. Author: Xiangrui Meng Closes #672 from mengxr/pyspark-mllib-util and squashes the following commits: 2943fa7 [Xiangrui Meng] format docs d61668d [Xiangrui Meng] add loadLibSVMFile and saveAsLibSVMFile to pyspark --- python/pyspark/mllib/linalg.py | 3 +- python/pyspark/mllib/util.py | 177 +++++++++++++++++++++++++++++++++ 2 files changed, 178 insertions(+), 2 deletions(-) create mode 100644 python/pyspark/mllib/util.py diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 0aa3a51de706b..7511ca7573ddb 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -49,8 +49,7 @@ def __init__(self, size, *args): >>> print SparseVector(4, [1, 3], [1.0, 5.5]) [1: 1.0, 3: 5.5] """ - assert type(size) == int, "first argument must be an int" - self.size = size + self.size = int(size) assert 1 <= len(args) <= 2, "must pass either 2 or 3 arguments" if len(args) == 1: pairs = args[0] diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py new file mode 100644 index 0000000000000..50d0cdd087625 --- /dev/null +++ b/python/pyspark/mllib/util.py @@ -0,0 +1,177 @@ +# +# 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. +# + +import numpy as np + +from pyspark.mllib.linalg import Vectors, SparseVector +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib._common import _convert_vector + +class MLUtils: + """ + Helper methods to load, save and pre-process data used in MLlib. + """ + + @staticmethod + def _parse_libsvm_line(line, multiclass): + """ + Parses a line in LIBSVM format into (label, indices, values). + """ + items = line.split(None) + label = float(items[0]) + if not multiclass: + label = 1.0 if label > 0.5 else 0.0 + nnz = len(items) - 1 + indices = np.zeros(nnz, dtype=np.int32) + values = np.zeros(nnz) + for i in xrange(nnz): + index, value = items[1 + i].split(":") + indices[i] = int(index) - 1 + values[i] = float(value) + return label, indices, values + + + @staticmethod + def _convert_labeled_point_to_libsvm(p): + """Converts a LabeledPoint to a string in LIBSVM format.""" + items = [str(p.label)] + v = _convert_vector(p.features) + if type(v) == np.ndarray: + for i in xrange(len(v)): + items.append(str(i + 1) + ":" + str(v[i])) + elif type(v) == SparseVector: + nnz = len(v.indices) + for i in xrange(nnz): + items.append(str(v.indices[i] + 1) + ":" + str(v.values[i])) + else: + raise TypeError("_convert_labeled_point_to_libsvm needs either ndarray or SparseVector" + " but got " % type(v)) + return " ".join(items) + + + @staticmethod + def loadLibSVMFile(sc, path, multiclass=False, numFeatures=-1, minPartitions=None): + """ + Loads labeled data in the LIBSVM format into an RDD of + LabeledPoint. The LIBSVM format is a text-based format used by + LIBSVM and LIBLINEAR. Each line represents a labeled sparse + feature vector using the following format: + + label index1:value1 index2:value2 ... + + where the indices are one-based and in ascending order. This + method parses each line into a LabeledPoint, where the feature + indices are converted to zero-based. + + @param sc: Spark context + @param path: file or directory path in any Hadoop-supported file + system URI + @param multiclass: whether the input labels contain more than + two classes. If false, any label with value + greater than 0.5 will be mapped to 1.0, or + 0.0 otherwise. So it works for both +1/-1 and + 1/0 cases. If true, the double value parsed + directly from the label string will be used + as the label value. + @param numFeatures: number of features, which will be determined + from the input data if a nonpositive value + is given. This is useful when the dataset is + already split into multiple files and you + want to load them separately, because some + features may not present in certain files, + which leads to inconsistent feature + dimensions. + @param minPartitions: min number of partitions + @return: labeled data stored as an RDD of LabeledPoint + + >>> from tempfile import NamedTemporaryFile + >>> from pyspark.mllib.util import MLUtils + >>> tempFile = NamedTemporaryFile(delete=True) + >>> tempFile.write("+1 1:1.0 3:2.0 5:3.0\\n-1\\n-1 2:4.0 4:5.0 6:6.0") + >>> tempFile.flush() + >>> examples = MLUtils.loadLibSVMFile(sc, tempFile.name).collect() + >>> multiclass_examples = MLUtils.loadLibSVMFile(sc, tempFile.name, True).collect() + >>> tempFile.close() + >>> examples[0].label + 1.0 + >>> examples[0].features.size + 6 + >>> print examples[0].features + [0: 1.0, 2: 2.0, 4: 3.0] + >>> examples[1].label + 0.0 + >>> examples[1].features.size + 6 + >>> print examples[1].features + [] + >>> examples[2].label + 0.0 + >>> examples[2].features.size + 6 + >>> print examples[2].features + [1: 4.0, 3: 5.0, 5: 6.0] + >>> multiclass_examples[1].label + -1.0 + """ + + lines = sc.textFile(path, minPartitions) + parsed = lines.map(lambda l: MLUtils._parse_libsvm_line(l, multiclass)) + if numFeatures <= 0: + parsed.cache() + numFeatures = parsed.map(lambda x: 0 if x[1].size == 0 else x[1][-1]).reduce(max) + 1 + return parsed.map(lambda x: LabeledPoint(x[0], Vectors.sparse(numFeatures, x[1], x[2]))) + + + @staticmethod + def saveAsLibSVMFile(data, dir): + """ + Save labeled data in LIBSVM format. + + @param data: an RDD of LabeledPoint to be saved + @param dir: directory to save the data + + >>> from tempfile import NamedTemporaryFile + >>> from fileinput import input + >>> from glob import glob + >>> from pyspark.mllib.util import MLUtils + >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])), \ + LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))] + >>> tempFile = NamedTemporaryFile(delete=True) + >>> tempFile.close() + >>> MLUtils.saveAsLibSVMFile(sc.parallelize(examples), tempFile.name) + >>> ''.join(sorted(input(glob(tempFile.name + "/part-0000*")))) + '0.0 1:1.01 2:2.02 3:3.03\\n1.1 1:1.23 3:4.56\\n' + """ + lines = data.map(lambda p: MLUtils._convert_labeled_point_to_libsvm(p)) + lines.saveAsTextFile(dir) + + +def _test(): + import doctest + from pyspark.context import SparkContext + globs = globals().copy() + # The small batch size here ensures that we see multiple batches, + # even in these small test examples: + globs['sc'] = SparkContext('local[2]', 'PythonTest', batchSize=2) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) + globs['sc'].stop() + if failure_count: + exit(-1) + + +if __name__ == "__main__": + _test() From 0c19bb161b9b2b96c0c55d3ea09e81fd798cbec0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?baishuo=28=E7=99=BD=E7=A1=95=29?= Date: Wed, 7 May 2014 16:02:55 -0700 Subject: [PATCH 276/641] Update GradientDescentSuite.scala MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit use more faster way to construct an array Author: baishuo(白硕) Closes #588 from baishuo/master and squashes the following commits: 45b95fb [baishuo(白硕)] Update GradientDescentSuite.scala c03b61c [baishuo(白硕)] Update GradientDescentSuite.scala b666d27 [baishuo(白硕)] Update GradientDescentSuite.scala --- .../spark/mllib/optimization/GradientDescentSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala index c4b433499a091..8a16284118cf7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala @@ -81,11 +81,11 @@ class GradientDescentSuite extends FunSuite with LocalSparkContext with ShouldMa // Add a extra variable consisting of all 1.0's for the intercept. val testData = GradientDescentSuite.generateGDInput(A, B, nPoints, 42) val data = testData.map { case LabeledPoint(label, features) => - label -> Vectors.dense(1.0, features.toArray: _*) + label -> Vectors.dense(1.0 +: features.toArray) } val dataRDD = sc.parallelize(data, 2).cache() - val initialWeightsWithIntercept = Vectors.dense(1.0, initialWeights: _*) + val initialWeightsWithIntercept = Vectors.dense(1.0 +: initialWeights.toArray) val (_, loss) = GradientDescent.runMiniBatchSGD( dataRDD, @@ -111,7 +111,7 @@ class GradientDescentSuite extends FunSuite with LocalSparkContext with ShouldMa // Add a extra variable consisting of all 1.0's for the intercept. val testData = GradientDescentSuite.generateGDInput(2.0, -1.5, 10000, 42) val data = testData.map { case LabeledPoint(label, features) => - label -> Vectors.dense(1.0, features.toArray: _*) + label -> Vectors.dense(1.0 +: features.toArray) } val dataRDD = sc.parallelize(data, 2).cache() From f269b016acb17b24d106dc2b32a1be389489bb01 Mon Sep 17 00:00:00 2001 From: Manish Amde Date: Wed, 7 May 2014 17:08:38 -0700 Subject: [PATCH 277/641] SPARK-1544 Add support for deep decision trees. @etrain and I came with a PR for arbitrarily deep decision trees at the cost of multiple passes over the data at deep tree levels. To summarize: 1) We take a parameter that indicates the amount of memory users want to reserve for computation on each worker (and 2x that at the driver). 2) Using that information, we calculate two things - the maximum depth to which we train as usual (which is, implicitly, the maximum number of nodes we want to train in parallel), and the size of the groups we should use in the case where we exceed this depth. cc: @atalwalkar, @hirakendu, @mengxr Author: Manish Amde Author: manishamde Author: Evan Sparks Closes #475 from manishamde/deep_tree and squashes the following commits: 968ca9d [Manish Amde] merged master 7fc9545 [Manish Amde] added docs ce004a1 [Manish Amde] minor formatting b27ad2c [Manish Amde] formatting 426bb28 [Manish Amde] programming guide blurb 8053fed [Manish Amde] more formatting 5eca9e4 [Manish Amde] grammar 4731cda [Manish Amde] formatting 5e82202 [Manish Amde] added documentation, fixed off by 1 error in max level calculation cbd9f14 [Manish Amde] modified scala.math to math dad9652 [Manish Amde] removed unused imports e0426ee [Manish Amde] renamed parameter 718506b [Manish Amde] added unit test 1517155 [Manish Amde] updated documentation 9dbdabe [Manish Amde] merge from master 719d009 [Manish Amde] updating user documentation fecf89a [manishamde] Merge pull request #6 from etrain/deep_tree 0287772 [Evan Sparks] Fixing scalastyle issue. 2f1e093 [Manish Amde] minor: added doc for maxMemory parameter 2f6072c [manishamde] Merge pull request #5 from etrain/deep_tree abc5a23 [Evan Sparks] Parameterizing max memory. 50b143a [Manish Amde] adding support for very deep trees --- docs/mllib-decision-tree.md | 15 +-- .../examples/mllib/DecisionTreeRunner.scala | 2 +- .../spark/mllib/tree/DecisionTree.scala | 103 ++++++++++++++++-- .../mllib/tree/configuration/Strategy.scala | 6 +- .../spark/mllib/tree/DecisionTreeSuite.scala | 84 ++++++++++++-- 5 files changed, 177 insertions(+), 33 deletions(-) diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 296277e58b341..acf0feff42a8d 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -93,17 +93,14 @@ The recursive tree construction is stopped at a node when one of the two conditi 1. The node depth is equal to the `maxDepth` training parameter 2. No split candidate leads to an information gain at the node. +### Max memory requirements + +For faster processing, the decision tree algorithm performs simultaneous histogram computations for all nodes at each level of the tree. This could lead to high memory requirements at deeper levels of the tree leading to memory overflow errors. To alleviate this problem, a 'maxMemoryInMB' training parameter is provided which specifies the maximum amount of memory at the workers (twice as much at the master) to be allocated to the histogram computation. The default value is conservatively chosen to be 128 MB to allow the decision algorithm to work in most scenarios. Once the memory requirements for a level-wise computation crosses the `maxMemoryInMB` threshold, the node training tasks at each subsequent level is split into smaller tasks. + ### Practical limitations -1. The tree implementation stores an `Array[Double]` of size *O(#features \* #splits \* 2^maxDepth)* - in memory for aggregating histograms over partitions. The current implementation might not scale - to very deep trees since the memory requirement grows exponentially with tree depth. -2. The implemented algorithm reads both sparse and dense data. However, it is not optimized for - sparse input. -3. Python is not supported in this release. - -We are planning to solve these problems in the near future. Please drop us a line if you encounter -any issues. +1. The implemented algorithm reads both sparse and dense data. However, it is not optimized for sparse input. +2. Python is not supported in this release. ## Examples diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 0bd847d7bab30..9832bec90d7ee 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -51,7 +51,7 @@ object DecisionTreeRunner { algo: Algo = Classification, maxDepth: Int = 5, impurity: ImpurityType = Gini, - maxBins: Int = 20) + maxBins: Int = 100) def main(args: Array[String]) { val defaultParams = Params() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 59ed01debf150..0fe30a3e7040b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -54,12 +54,13 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo // Find the splits and the corresponding bins (interval between the splits) using a sample // of the input data. val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) - logDebug("numSplits = " + bins(0).length) + val numBins = bins(0).length + logDebug("numBins = " + numBins) // depth of the decision tree val maxDepth = strategy.maxDepth // the max number of nodes possible given the depth of the tree - val maxNumNodes = scala.math.pow(2, maxDepth).toInt - 1 + val maxNumNodes = math.pow(2, maxDepth).toInt - 1 // Initialize an array to hold filters applied to points for each node. val filters = new Array[List[Filter]](maxNumNodes) // The filter at the top node is an empty list. @@ -68,7 +69,28 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo val parentImpurities = new Array[Double](maxNumNodes) // dummy value for top node (updated during first split calculation) val nodes = new Array[Node](maxNumNodes) + // num features + val numFeatures = input.take(1)(0).features.size + + // Calculate level for single group construction + // Max memory usage for aggregates + val maxMemoryUsage = strategy.maxMemoryInMB * 1024 * 1024 + logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.") + val numElementsPerNode = + strategy.algo match { + case Classification => 2 * numBins * numFeatures + case Regression => 3 * numBins * numFeatures + } + + logDebug("numElementsPerNode = " + numElementsPerNode) + val arraySizePerNode = 8 * numElementsPerNode // approx. memory usage for bin aggregate array + val maxNumberOfNodesPerGroup = math.max(maxMemoryUsage / arraySizePerNode, 1) + logDebug("maxNumberOfNodesPerGroup = " + maxNumberOfNodesPerGroup) + // nodes at a level is 2^level. level is zero indexed. + val maxLevelForSingleGroup = math.max( + (math.log(maxNumberOfNodesPerGroup) / math.log(2)).floor.toInt, 0) + logDebug("max level for single group = " + maxLevelForSingleGroup) /* * The main idea here is to perform level-wise training of the decision tree nodes thus @@ -88,7 +110,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo // Find best split for all nodes at a level. val splitsStatsForLevel = DecisionTree.findBestSplits(input, parentImpurities, strategy, - level, filters, splits, bins) + level, filters, splits, bins, maxLevelForSingleGroup) for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { // Extract info for nodes at the current level. @@ -98,7 +120,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo filters) logDebug("final best split = " + nodeSplitStats._1) } - require(scala.math.pow(2, level) == splitsStatsForLevel.length) + require(math.pow(2, level) == splitsStatsForLevel.length) // Check whether all the nodes at the current level at leaves. val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0) logDebug("all leaf = " + allLeaf) @@ -109,6 +131,10 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo } } + logDebug("#####################################") + logDebug("Extracting tree model") + logDebug("#####################################") + // Initialize the top or root node of the tree. val topNode = nodes(0) // Build the full tree using the node info calculated in the level-wise best split calculations. @@ -127,7 +153,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo nodes: Array[Node]): Unit = { val split = nodeSplitStats._1 val stats = nodeSplitStats._2 - val nodeIndex = scala.math.pow(2, level).toInt - 1 + index + val nodeIndex = math.pow(2, level).toInt - 1 + index val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth - 1) val node = new Node(nodeIndex, stats.predict, isLeaf, Some(split), None, None, Some(stats)) logDebug("Node = " + node) @@ -148,7 +174,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo var i = 0 while (i <= 1) { // Calculate the index of the node from the node level and the index at the current level. - val nodeIndex = scala.math.pow(2, level + 1).toInt - 1 + 2 * index + i + val nodeIndex = math.pow(2, level + 1).toInt - 1 + 2 * index + i if (level < maxDepth - 1) { val impurity = if (i == 0) { nodeSplitStats._2.leftImpurity @@ -249,7 +275,8 @@ object DecisionTree extends Serializable with Logging { private val InvalidBinIndex = -1 /** - * Returns an array of optimal splits for all nodes at a given level + * Returns an array of optimal splits for all nodes at a given level. Splits the task into + * multiple groups if the level-wise training task could lead to memory overflow. * * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data * for DecisionTree @@ -260,6 +287,7 @@ object DecisionTree extends Serializable with Logging { * @param filters Filters for all nodes at a given level * @param splits possible splits for all features * @param bins possible bins for all features + * @param maxLevelForSingleGroup the deepest level for single-group level-wise computation. * @return array of splits with best splits for all nodes at a given level. */ protected[tree] def findBestSplits( @@ -269,7 +297,57 @@ object DecisionTree extends Serializable with Logging { level: Int, filters: Array[List[Filter]], splits: Array[Array[Split]], - bins: Array[Array[Bin]]): Array[(Split, InformationGainStats)] = { + bins: Array[Array[Bin]], + maxLevelForSingleGroup: Int): Array[(Split, InformationGainStats)] = { + // split into groups to avoid memory overflow during aggregation + if (level > maxLevelForSingleGroup) { + // When information for all nodes at a given level cannot be stored in memory, + // the nodes are divided into multiple groups at each level with the number of groups + // increasing exponentially per level. For example, if maxLevelForSingleGroup is 10, + // numGroups is equal to 2 at level 11 and 4 at level 12, respectively. + val numGroups = math.pow(2, (level - maxLevelForSingleGroup)).toInt + logDebug("numGroups = " + numGroups) + var bestSplits = new Array[(Split, InformationGainStats)](0) + // Iterate over each group of nodes at a level. + var groupIndex = 0 + while (groupIndex < numGroups) { + val bestSplitsForGroup = findBestSplitsPerGroup(input, parentImpurities, strategy, level, + filters, splits, bins, numGroups, groupIndex) + bestSplits = Array.concat(bestSplits, bestSplitsForGroup) + groupIndex += 1 + } + bestSplits + } else { + findBestSplitsPerGroup(input, parentImpurities, strategy, level, filters, splits, bins) + } + } + + /** + * Returns an array of optimal splits for a group of nodes at a given level + * + * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data + * for DecisionTree + * @param parentImpurities Impurities for all parent nodes for the current level + * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing + * parameters for construction the DecisionTree + * @param level Level of the tree + * @param filters Filters for all nodes at a given level + * @param splits possible splits for all features + * @param bins possible bins for all features + * @param numGroups total number of node groups at the current level. Default value is set to 1. + * @param groupIndex index of the node group being processed. Default value is set to 0. + * @return array of splits with best splits for all nodes at a given level. + */ + private def findBestSplitsPerGroup( + input: RDD[LabeledPoint], + parentImpurities: Array[Double], + strategy: Strategy, + level: Int, + filters: Array[List[Filter]], + splits: Array[Array[Split]], + bins: Array[Array[Bin]], + numGroups: Int = 1, + groupIndex: Int = 0): Array[(Split, InformationGainStats)] = { /* * The high-level description for the best split optimizations are noted here. @@ -296,7 +374,7 @@ object DecisionTree extends Serializable with Logging { */ // common calculations for multiple nested methods - val numNodes = scala.math.pow(2, level).toInt + val numNodes = math.pow(2, level).toInt / numGroups logDebug("numNodes = " + numNodes) // Find the number of features by looking at the first sample. val numFeatures = input.first().features.size @@ -304,12 +382,15 @@ object DecisionTree extends Serializable with Logging { val numBins = bins(0).length logDebug("numBins = " + numBins) + // shift when more than one group is used at deep tree level + val groupShift = numNodes * groupIndex + /** Find the filters used before reaching the current code. */ def findParentFilters(nodeIndex: Int): List[Filter] = { if (level == 0) { List[Filter]() } else { - val nodeFilterIndex = scala.math.pow(2, level).toInt - 1 + nodeIndex + val nodeFilterIndex = math.pow(2, level).toInt - 1 + nodeIndex + groupShift filters(nodeFilterIndex) } } @@ -878,7 +959,7 @@ object DecisionTree extends Serializable with Logging { // Iterating over all nodes at this level var node = 0 while (node < numNodes) { - val nodeImpurityIndex = scala.math.pow(2, level).toInt - 1 + node + val nodeImpurityIndex = math.pow(2, level).toInt - 1 + node + groupShift val binsForNode: Array[Double] = getBinDataForNode(node) logDebug("nodeImpurityIndex = " + nodeImpurityIndex) val parentNodeImpurity = parentImpurities(nodeImpurityIndex) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 8767aca47cd5a..1b505fd76eb75 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -35,6 +35,9 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * k) implies the feature n is categorical with k categories 0, * 1, 2, ... , k-1. It's important to note that features are * zero-indexed. + * @param maxMemoryInMB maximum memory in MB allocated to histogram aggregation. Default value is + * 128 MB. + * */ @Experimental class Strategy ( @@ -43,4 +46,5 @@ class Strategy ( val maxDepth: Int, val maxBins: Int = 100, val quantileCalculationStrategy: QuantileStrategy = Sort, - val categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int]()) extends Serializable + val categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), + val maxMemoryInMB: Int = 128) extends Serializable diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index be383aab714d3..35e92d71dc63f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -22,7 +22,8 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} import org.apache.spark.mllib.tree.model.Filter -import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.model.Split +import org.apache.spark.mllib.tree.configuration.{FeatureType, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.linalg.Vectors @@ -242,7 +243,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) val bestSplits = DecisionTree.findBestSplits(rdd, new Array(7), strategy, 0, - Array[List[Filter]](), splits, bins) + Array[List[Filter]](), splits, bins, 10) val split = bestSplits(0)._1 assert(split.categories.length === 1) @@ -269,7 +270,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) val (splits, bins) = DecisionTree.findSplitsBins(rdd,strategy) val bestSplits = DecisionTree.findBestSplits(rdd, new Array(7), strategy, 0, - Array[List[Filter]](), splits, bins) + Array[List[Filter]](), splits, bins, 10) val split = bestSplits(0)._1 assert(split.categories.length === 1) @@ -298,7 +299,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val bestSplits = DecisionTree.findBestSplits(rdd, new Array(7), strategy, 0, - Array[List[Filter]](), splits, bins) + Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) assert(bestSplits(0)._1.threshold === 10) @@ -321,7 +322,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val bestSplits = DecisionTree.findBestSplits(rdd, Array(0.0), strategy, 0, - Array[List[Filter]](), splits, bins) + Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) assert(bestSplits(0)._1.threshold === 10) @@ -345,7 +346,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val bestSplits = DecisionTree.findBestSplits(rdd, Array(0.0), strategy, 0, - Array[List[Filter]](), splits, bins) + Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) assert(bestSplits(0)._1.threshold === 10) @@ -369,7 +370,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val bestSplits = DecisionTree.findBestSplits(rdd, Array(0.0), strategy, 0, - Array[List[Filter]](), splits, bins) + Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) assert(bestSplits(0)._1.threshold === 10) @@ -378,13 +379,60 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.rightImpurity === 0) assert(bestSplits(0)._2.predict === 1) } + + test("test second level node building with/without groups") { + val arr = DecisionTreeSuite.generateOrderedLabeledPoints() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy(Classification, Entropy, 3, 100) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + assert(splits.length === 2) + assert(splits(0).length === 99) + assert(bins.length === 2) + assert(bins(0).length === 100) + assert(splits(0).length === 99) + assert(bins(0).length === 100) + + val leftFilter = Filter(new Split(0, 400, FeatureType.Continuous, List()), -1) + val rightFilter = Filter(new Split(0, 400, FeatureType.Continuous, List()) ,1) + val filters = Array[List[Filter]](List(), List(leftFilter), List(rightFilter)) + val parentImpurities = Array(0.5, 0.5, 0.5) + + // Single group second level tree construction. + val bestSplits = DecisionTree.findBestSplits(rdd, parentImpurities, strategy, 1, filters, + splits, bins, 10) + assert(bestSplits.length === 2) + assert(bestSplits(0)._2.gain > 0) + assert(bestSplits(1)._2.gain > 0) + + // maxLevelForSingleGroup parameter is set to 0 to force splitting into groups for second + // level tree construction. + val bestSplitsWithGroups = DecisionTree.findBestSplits(rdd, parentImpurities, strategy, 1, + filters, splits, bins, 0) + assert(bestSplitsWithGroups.length === 2) + assert(bestSplitsWithGroups(0)._2.gain > 0) + assert(bestSplitsWithGroups(1)._2.gain > 0) + + // Verify whether the splits obtained using single group and multiple group level + // construction strategies are the same. + for (i <- 0 until bestSplits.length) { + assert(bestSplits(i)._1 === bestSplitsWithGroups(i)._1) + assert(bestSplits(i)._2.gain === bestSplitsWithGroups(i)._2.gain) + assert(bestSplits(i)._2.impurity === bestSplitsWithGroups(i)._2.impurity) + assert(bestSplits(i)._2.leftImpurity === bestSplitsWithGroups(i)._2.leftImpurity) + assert(bestSplits(i)._2.rightImpurity === bestSplitsWithGroups(i)._2.rightImpurity) + assert(bestSplits(i)._2.predict === bestSplitsWithGroups(i)._2.predict) + } + + } + } object DecisionTreeSuite { def generateOrderedLabeledPointsWithLabel0(): Array[LabeledPoint] = { val arr = new Array[LabeledPoint](1000) - for (i <- 0 until 1000){ + for (i <- 0 until 1000) { val lp = new LabeledPoint(0.0, Vectors.dense(i.toDouble, 1000.0 - i)) arr(i) = lp } @@ -393,17 +441,31 @@ object DecisionTreeSuite { def generateOrderedLabeledPointsWithLabel1(): Array[LabeledPoint] = { val arr = new Array[LabeledPoint](1000) - for (i <- 0 until 1000){ + for (i <- 0 until 1000) { val lp = new LabeledPoint(1.0, Vectors.dense(i.toDouble, 999.0 - i)) arr(i) = lp } arr } + def generateOrderedLabeledPoints(): Array[LabeledPoint] = { + val arr = new Array[LabeledPoint](1000) + for (i <- 0 until 1000) { + if (i < 600) { + val lp = new LabeledPoint(0.0, Vectors.dense(i.toDouble, 1000.0 - i)) + arr(i) = lp + } else { + val lp = new LabeledPoint(1.0, Vectors.dense(i.toDouble, 1000.0 - i)) + arr(i) = lp + } + } + arr + } + def generateCategoricalDataPoints(): Array[LabeledPoint] = { val arr = new Array[LabeledPoint](1000) - for (i <- 0 until 1000){ - if (i < 600){ + for (i <- 0 until 1000) { + if (i < 600) { arr(i) = new LabeledPoint(1.0, Vectors.dense(0.0, 1.0)) } else { arr(i) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0)) From 108c4c16cc82af2e161d569d2c23849bdbf4aadb Mon Sep 17 00:00:00 2001 From: Sandeep Date: Thu, 8 May 2014 00:15:05 -0400 Subject: [PATCH 278/641] SPARK-1668: Add implicit preference as an option to examples/MovieLensALS Add --implicitPrefs as an command-line option to the example app MovieLensALS under examples/ Author: Sandeep Closes #597 from techaddict/SPARK-1668 and squashes the following commits: 8b371dc [Sandeep] Second Pass on reviews by mengxr eca9d37 [Sandeep] based on mengxr's suggestions 937e54c [Sandeep] Changes 5149d40 [Sandeep] Changes based on review 1dd7657 [Sandeep] use mean() 42444d7 [Sandeep] Based on Suggestions by mengxr e3082fa [Sandeep] SPARK-1668: Add implicit preference as an option to examples/MovieLensALS Add --implicitPrefs as an command-line option to the example app MovieLensALS under examples/ --- .../spark/examples/mllib/MovieLensALS.scala | 55 ++++++++++++++++--- 1 file changed, 46 insertions(+), 9 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala index 703f02255b94b..0e4447e0de24f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala @@ -43,7 +43,8 @@ object MovieLensALS { kryo: Boolean = false, numIterations: Int = 20, lambda: Double = 1.0, - rank: Int = 10) + rank: Int = 10, + implicitPrefs: Boolean = false) def main(args: Array[String]) { val defaultParams = Params() @@ -62,6 +63,9 @@ object MovieLensALS { opt[Unit]("kryo") .text(s"use Kryo serialization") .action((_, c) => c.copy(kryo = true)) + opt[Unit]("implicitPrefs") + .text("use implicit preference") + .action((_, c) => c.copy(implicitPrefs = true)) arg[String]("") .required() .text("input paths to a MovieLens dataset of ratings") @@ -88,7 +92,25 @@ object MovieLensALS { val ratings = sc.textFile(params.input).map { line => val fields = line.split("::") - Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble) + if (params.implicitPrefs) { + /* + * MovieLens ratings are on a scale of 1-5: + * 5: Must see + * 4: Will enjoy + * 3: It's okay + * 2: Fairly bad + * 1: Awful + * So we should not recommend a movie if the predicted rating is less than 3. + * To map ratings to confidence scores, we use + * 5 -> 2.5, 4 -> 1.5, 3 -> 0.5, 2 -> -0.5, 1 -> -1.5. This mappings means unobserved + * entries are generally between It's okay and Fairly bad. + * The semantics of 0 in this expanded world of non-positive weights + * are "the same as never having interacted at all". + */ + Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble - 2.5) + } else { + Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble) + } }.cache() val numRatings = ratings.count() @@ -99,7 +121,18 @@ object MovieLensALS { val splits = ratings.randomSplit(Array(0.8, 0.2)) val training = splits(0).cache() - val test = splits(1).cache() + val test = if (params.implicitPrefs) { + /* + * 0 means "don't know" and positive values mean "confident that the prediction should be 1". + * Negative values means "confident that the prediction should be 0". + * We have in this case used some kind of weighted RMSE. The weight is the absolute value of + * the confidence. The error is the difference between prediction and either 1 or 0, + * depending on whether r is positive or negative. + */ + splits(1).map(x => Rating(x.user, x.product, if (x.rating > 0) 1.0 else 0.0)) + } else { + splits(1) + }.cache() val numTraining = training.count() val numTest = test.count() @@ -111,9 +144,10 @@ object MovieLensALS { .setRank(params.rank) .setIterations(params.numIterations) .setLambda(params.lambda) + .setImplicitPrefs(params.implicitPrefs) .run(training) - val rmse = computeRmse(model, test, numTest) + val rmse = computeRmse(model, test, params.implicitPrefs) println(s"Test RMSE = $rmse.") @@ -121,11 +155,14 @@ object MovieLensALS { } /** Compute RMSE (Root Mean Squared Error). */ - def computeRmse(model: MatrixFactorizationModel, data: RDD[Rating], n: Long) = { + def computeRmse(model: MatrixFactorizationModel, data: RDD[Rating], implicitPrefs: Boolean) = { + + def mapPredictedRating(r: Double) = if (implicitPrefs) math.max(math.min(r, 1.0), 0.0) else r + val predictions: RDD[Rating] = model.predict(data.map(x => (x.user, x.product))) - val predictionsAndRatings = predictions.map(x => ((x.user, x.product), x.rating)) - .join(data.map(x => ((x.user, x.product), x.rating))) - .values - math.sqrt(predictionsAndRatings.map(x => (x._1 - x._2) * (x._1 - x._2)).reduce(_ + _) / n) + val predictionsAndRatings = predictions.map{ x => + ((x.user, x.product), mapPredictedRating(x.rating)) + }.join(data.map(x => ((x.user, x.product), x.rating))).values + math.sqrt(predictionsAndRatings.map(x => (x._1 - x._2) * (x._1 - x._2)).mean()) } } From 6ed7e2cd01955adfbb3960e2986b6d19eaee8717 Mon Sep 17 00:00:00 2001 From: Evan Sparks Date: Thu, 8 May 2014 00:24:36 -0400 Subject: [PATCH 279/641] Use numpy directly for matrix multiply. Using matrix multiply to compute XtX and XtY yields a 5-20x speedup depending on problem size. For example - the following takes 19s locally after this change vs. 5m21s before the change. (16x speedup). bin/pyspark examples/src/main/python/als.py local[8] 1000 1000 50 10 10 Author: Evan Sparks Closes #687 from etrain/patch-1 and squashes the following commits: e094dbc [Evan Sparks] Touching only diaganols on update. d1ab9b6 [Evan Sparks] Use numpy directly for matrix multiply. --- examples/src/main/python/als.py | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index a77dfb2577835..33700ab4f8c53 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -36,14 +36,13 @@ def rmse(R, ms, us): def update(i, vec, mat, ratings): uu = mat.shape[0] ff = mat.shape[1] - XtX = matrix(np.zeros((ff, ff))) - Xty = np.zeros((ff, 1)) - - for j in range(uu): - v = mat[j, :] - XtX += v.T * v - Xty += v.T * ratings[i, j] - XtX += np.eye(ff, ff) * LAMBDA * uu + + XtX = mat.T * mat + XtY = mat.T * ratings[i, :].T + + for j in range(ff): + XtX[j,j] += LAMBDA * uu + return np.linalg.solve(XtX, Xty) if __name__ == "__main__": From 19c8fb02bc2c2f76c3c45bfff4b8d093be9d7c66 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 8 May 2014 01:08:43 -0400 Subject: [PATCH 280/641] [SQL] Improve SparkSQL Aggregates * Add native min/max (was using hive before). * Handle nulls correctly in Avg and Sum. Author: Michael Armbrust Closes #683 from marmbrus/aggFixes and squashes the following commits: 64fe30b [Michael Armbrust] Improve SparkSQL Aggregates * Add native min/max (was using hive before). * Handle nulls correctly in Avg and Sum. --- .../apache/spark/sql/catalyst/SqlParser.scala | 4 + .../sql/catalyst/expressions/aggregates.scala | 85 ++++++++++++++++--- .../org/apache/spark/sql/SQLQuerySuite.scala | 7 ++ .../scala/org/apache/spark/sql/TestData.scala | 10 +++ 4 files changed, 96 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 8c76a3aa96546..b3a3a1ef1b5eb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -114,6 +114,8 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val JOIN = Keyword("JOIN") protected val LEFT = Keyword("LEFT") protected val LIMIT = Keyword("LIMIT") + protected val MAX = Keyword("MAX") + protected val MIN = Keyword("MIN") protected val NOT = Keyword("NOT") protected val NULL = Keyword("NULL") protected val ON = Keyword("ON") @@ -318,6 +320,8 @@ class SqlParser extends StandardTokenParsers with PackratParsers { COUNT ~> "(" ~> DISTINCT ~> expression <~ ")" ^^ { case exp => CountDistinct(exp :: Nil) } | FIRST ~> "(" ~> expression <~ ")" ^^ { case exp => First(exp) } | AVG ~> "(" ~> expression <~ ")" ^^ { case exp => Average(exp) } | + MIN ~> "(" ~> expression <~ ")" ^^ { case exp => Min(exp) } | + MAX ~> "(" ~> expression <~ ")" ^^ { case exp => Max(exp) } | IF ~> "(" ~> expression ~ "," ~ expression ~ "," ~ expression <~ ")" ^^ { case c ~ "," ~ t ~ "," ~ f => If(c,t,f) } | diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index b152f95f96c70..7777d372903e2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -86,6 +86,67 @@ abstract class AggregateFunction override def newInstance() = makeCopy(productIterator.map { case a: AnyRef => a }.toArray) } +case class Min(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { + override def references = child.references + override def nullable = child.nullable + override def dataType = child.dataType + override def toString = s"MIN($child)" + + override def asPartial: SplitEvaluation = { + val partialMin = Alias(Min(child), "PartialMin")() + SplitEvaluation(Min(partialMin.toAttribute), partialMin :: Nil) + } + + override def newInstance() = new MinFunction(child, this) +} + +case class MinFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + + var currentMin: Any = _ + + override def update(input: Row): Unit = { + if (currentMin == null) { + currentMin = expr.eval(input) + } else if(GreaterThan(Literal(currentMin, expr.dataType), expr).eval(input) == true) { + currentMin = expr.eval(input) + } + } + + override def eval(input: Row): Any = currentMin +} + +case class Max(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { + override def references = child.references + override def nullable = child.nullable + override def dataType = child.dataType + override def toString = s"MAX($child)" + + override def asPartial: SplitEvaluation = { + val partialMax = Alias(Max(child), "PartialMax")() + SplitEvaluation(Max(partialMax.toAttribute), partialMax :: Nil) + } + + override def newInstance() = new MaxFunction(child, this) +} + +case class MaxFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + + var currentMax: Any = _ + + override def update(input: Row): Unit = { + if (currentMax == null) { + currentMax = expr.eval(input) + } else if(LessThan(Literal(currentMax, expr.dataType), expr).eval(input) == true) { + currentMax = expr.eval(input) + } + } + + override def eval(input: Row): Any = currentMax +} + + case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { override def references = child.references override def nullable = false @@ -97,7 +158,7 @@ case class Count(child: Expression) extends PartialAggregate with trees.UnaryNod SplitEvaluation(Sum(partialCount.toAttribute), partialCount :: Nil) } - override def newInstance()= new CountFunction(child, this) + override def newInstance() = new CountFunction(child, this) } case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpression { @@ -106,7 +167,7 @@ case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpressi override def nullable = false override def dataType = IntegerType override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")}})" - override def newInstance()= new CountDistinctFunction(expressions, this) + override def newInstance() = new CountDistinctFunction(expressions, this) } case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -126,7 +187,7 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN partialCount :: partialSum :: Nil) } - override def newInstance()= new AverageFunction(child, this) + override def newInstance() = new AverageFunction(child, this) } case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -142,7 +203,7 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ partialSum :: Nil) } - override def newInstance()= new SumFunction(child, this) + override def newInstance() = new SumFunction(child, this) } case class SumDistinct(child: Expression) @@ -153,7 +214,7 @@ case class SumDistinct(child: Expression) override def dataType = child.dataType override def toString = s"SUM(DISTINCT $child)" - override def newInstance()= new SumDistinctFunction(child, this) + override def newInstance() = new SumDistinctFunction(child, this) } case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -168,7 +229,7 @@ case class First(child: Expression) extends PartialAggregate with trees.UnaryNod First(partialFirst.toAttribute), partialFirst :: Nil) } - override def newInstance()= new FirstFunction(child, this) + override def newInstance() = new FirstFunction(child, this) } case class AverageFunction(expr: Expression, base: AggregateExpression) @@ -176,11 +237,13 @@ case class AverageFunction(expr: Expression, base: AggregateExpression) def this() = this(null, null) // Required for serialization. + private val zero = Cast(Literal(0), expr.dataType) + private var count: Long = _ - private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).eval(EmptyRow)) + private val sum = MutableLiteral(zero.eval(EmptyRow)) private val sumAsDouble = Cast(sum, DoubleType) - private val addFunction = Add(sum, expr) + private val addFunction = Add(sum, Coalesce(Seq(expr, zero))) override def eval(input: Row): Any = sumAsDouble.eval(EmptyRow).asInstanceOf[Double] / count.toDouble @@ -209,9 +272,11 @@ case class CountFunction(expr: Expression, base: AggregateExpression) extends Ag case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. - private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).eval(null)) + private val zero = Cast(Literal(0), expr.dataType) + + private val sum = MutableLiteral(zero.eval(null)) - private val addFunction = Add(sum, expr) + private val addFunction = Add(sum, Coalesce(Seq(expr, zero))) override def update(input: Row): Unit = { sum.update(addFunction, input) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index dde957d715a28..e966d89c30cf5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -50,6 +50,13 @@ class SQLQuerySuite extends QueryTest { Seq((1,3),(2,3),(3,3))) } + test("aggregates with nulls") { + checkAnswer( + sql("SELECT MIN(a), MAX(a), AVG(a), SUM(a), COUNT(a) FROM nullInts"), + (1, 3, 2, 6, 3) :: Nil + ) + } + test("select *") { checkAnswer( sql("SELECT * FROM testData"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index b5973c0f51be8..aa71e274f7f4b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -84,4 +84,14 @@ object TestData { List.fill(2)(StringData(null)) ++ List.fill(2)(StringData("test"))) nullableRepeatedData.registerAsTable("nullableRepeatedData") + + case class NullInts(a: Integer) + val nullInts = + TestSQLContext.sparkContext.parallelize( + NullInts(1) :: + NullInts(2) :: + NullInts(3) :: + NullInts(null) :: Nil + ) + nullInts.registerAsTable("nullInts") } From 44dd57fb66bb676d753ad8d9757f9f4c03364113 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 8 May 2014 10:23:05 -0700 Subject: [PATCH 281/641] SPARK-1565, update examples to be used with spark-submit script. Commit for initial feedback, basically I am curious if we should prompt user for providing args esp. when its mandatory. And can we skip if they are not ? Also few other things that did not work like `bin/spark-submit examples/target/scala-2.10/spark-examples-1.0.0-SNAPSHOT-hadoop1.0.4.jar --class org.apache.spark.examples.SparkALS --arg 100 500 10 5 2` Not all the args get passed properly, may be I have messed up something will try to sort it out hopefully. Author: Prashant Sharma Closes #552 from ScrapCodes/SPARK-1565/update-examples and squashes the following commits: 669dd23 [Prashant Sharma] Review comments 2727e70 [Prashant Sharma] SPARK-1565, update examples to be used with spark-submit script. --- .gitignore | 1 + .../scala/org/apache/spark/SparkContext.scala | 8 ++-- .../org/apache/spark/examples/JavaHdfsLR.java | 13 ++++--- .../apache/spark/examples/JavaLogQuery.java | 13 +++---- .../apache/spark/examples/JavaPageRank.java | 15 +++++--- .../apache/spark/examples/JavaSparkPi.java | 18 ++++----- .../org/apache/spark/examples/JavaTC.java | 24 ++++++------ .../apache/spark/examples/JavaWordCount.java | 12 +++--- .../apache/spark/examples/mllib/JavaALS.java | 22 +++++------ .../spark/examples/mllib/JavaKMeans.java | 22 +++++------ .../apache/spark/examples/mllib/JavaLR.java | 18 ++++----- .../spark/examples/sql/JavaSparkSQL.java | 5 ++- .../streaming/JavaFlumeEventCount.java | 19 ++++------ .../streaming/JavaKafkaWordCount.java | 27 +++++++------- .../streaming/JavaNetworkWordCount.java | 25 ++++++------- .../examples/streaming/JavaQueueStream.java | 22 +++++------ .../apache/spark/examples/BroadcastTest.scala | 22 +++++------ .../spark/examples/CassandraCQLTest.scala | 19 +++++----- .../apache/spark/examples/CassandraTest.scala | 10 ++--- .../examples/ExceptionHandlingTest.scala | 11 ++---- .../apache/spark/examples/GroupByTest.scala | 25 ++++++------- .../org/apache/spark/examples/HBaseTest.scala | 6 +-- .../org/apache/spark/examples/HdfsTest.scala | 4 +- .../org/apache/spark/examples/LogQuery.scala | 14 +++---- .../spark/examples/MultiBroadcastTest.scala | 17 ++++----- .../examples/SimpleSkewedGroupByTest.scala | 24 ++++++------ .../spark/examples/SkewedGroupByTest.scala | 25 ++++++------- .../org/apache/spark/examples/SparkALS.scala | 18 +++------ .../apache/spark/examples/SparkHdfsLR.scala | 13 ++++--- .../apache/spark/examples/SparkKMeans.scala | 18 ++++----- .../org/apache/spark/examples/SparkLR.scala | 11 ++---- .../apache/spark/examples/SparkPageRank.scala | 14 +++---- .../org/apache/spark/examples/SparkPi.scala | 10 ++--- .../org/apache/spark/examples/SparkTC.scala | 12 ++---- .../spark/examples/SparkTachyonHdfsLR.scala | 12 ++---- .../spark/examples/SparkTachyonPi.scala | 10 ++--- .../examples/bagel/WikipediaPageRank.scala | 10 ++--- .../bagel/WikipediaPageRankStandalone.scala | 10 ++--- .../examples/graphx/LiveJournalPageRank.scala | 6 +-- .../spark/examples/sql/RDDRelation.scala | 5 ++- .../examples/sql/hive/HiveFromSpark.scala | 5 ++- .../examples/streaming/ActorWordCount.scala | 21 +++++------ .../examples/streaming/FlumeEventCount.scala | 14 +++---- .../examples/streaming/HdfsWordCount.scala | 18 ++++----- .../examples/streaming/KafkaWordCount.scala | 21 +++++------ .../examples/streaming/MQTTWordCount.scala | 26 ++++++------- .../examples/streaming/NetworkWordCount.scala | 23 +++++------- .../examples/streaming/QueueStream.scala | 10 ++--- .../examples/streaming/RawNetworkGrep.scala | 16 ++++---- .../RecoverableNetworkWordCount.scala | 37 ++++++++++--------- .../streaming/StatefulNetworkWordCount.scala | 21 +++++------ .../streaming/TwitterAlgebirdCMS.scala | 15 +++----- .../streaming/TwitterAlgebirdHLL.scala | 14 +++---- .../streaming/TwitterPopularTags.scala | 13 ++----- .../examples/streaming/ZeroMQWordCount.scala | 23 ++++++------ .../apache/spark/graphx/lib/Analytics.scala | 18 +++++---- 56 files changed, 405 insertions(+), 480 deletions(-) diff --git a/.gitignore b/.gitignore index 32b603f1bc84f..ad72588b472d6 100644 --- a/.gitignore +++ b/.gitignore @@ -49,6 +49,7 @@ unit-tests.log /lib/ rat-results.txt scalastyle.txt +conf/*.conf # For Hive metastore_db/ diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index eb14d87467af7..9d7c2c8d3d630 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -74,10 +74,10 @@ class SparkContext(config: SparkConf) extends Logging { * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] * from a list of input files or InputFormats for the application. */ - @DeveloperApi - def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = { - this(config) - this.preferredNodeLocationData = preferredNodeLocationData + @DeveloperApi + def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = { + this(config) + this.preferredNodeLocationData = preferredNodeLocationData } /** diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index bd96274021756..6c177de359b60 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -17,6 +17,7 @@ package org.apache.spark.examples; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; @@ -103,16 +104,16 @@ public static void printWeights(double[] a) { public static void main(String[] args) { - if (args.length < 3) { - System.err.println("Usage: JavaHdfsLR "); + if (args.length < 2) { + System.err.println("Usage: JavaHdfsLR "); System.exit(1); } - JavaSparkContext sc = new JavaSparkContext(args[0], "JavaHdfsLR", - System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaHdfsLR.class)); - JavaRDD lines = sc.textFile(args[1]); + SparkConf sparkConf = new SparkConf().setAppName("JavaHdfsLR"); + JavaSparkContext sc = new JavaSparkContext(sparkConf); + JavaRDD lines = sc.textFile(args[0]); JavaRDD points = lines.map(new ParsePoint()).cache(); - int ITERATIONS = Integer.parseInt(args[2]); + int ITERATIONS = Integer.parseInt(args[1]); // Initialize w to a random value double[] w = new double[D]; 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 3f7a879538016..812e9d5580cbf 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -20,6 +20,7 @@ import com.google.common.collect.Lists; import scala.Tuple2; import scala.Tuple3; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -34,6 +35,8 @@ /** * Executes a roll up-style query against Apache logs. + * + * Usage: JavaLogQuery [logFile] */ public final class JavaLogQuery { @@ -97,15 +100,11 @@ public static Stats extractStats(String line) { } public static void main(String[] args) { - if (args.length == 0) { - System.err.println("Usage: JavaLogQuery [logFile]"); - System.exit(1); - } - JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", - System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaLogQuery.class)); + SparkConf sparkConf = new SparkConf().setAppName("JavaLogQuery"); + JavaSparkContext jsc = new JavaSparkContext(sparkConf); - JavaRDD dataSet = (args.length == 2) ? jsc.textFile(args[1]) : jsc.parallelize(exampleApacheLogs); + JavaRDD dataSet = (args.length == 1) ? jsc.textFile(args[0]) : jsc.parallelize(exampleApacheLogs); JavaPairRDD, Stats> extracted = dataSet.mapToPair(new PairFunction, Stats>() { @Override 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 e31f676f5fd4c..7ea6df9c17245 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -18,9 +18,12 @@ package org.apache.spark.examples; + import scala.Tuple2; import com.google.common.collect.Iterables; + +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -54,20 +57,20 @@ public Double call(Double a, Double b) { } public static void main(String[] args) throws Exception { - if (args.length < 3) { - System.err.println("Usage: JavaPageRank "); + if (args.length < 2) { + System.err.println("Usage: JavaPageRank "); System.exit(1); } - JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaPageRank", - System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaPageRank.class)); + SparkConf sparkConf = new SparkConf().setAppName("JavaPageRank"); + JavaSparkContext ctx = new JavaSparkContext(sparkConf); // Loads in input file. It should be in format of: // URL neighbor URL // URL neighbor URL // URL neighbor URL // ... - JavaRDD lines = ctx.textFile(args[1], 1); + JavaRDD lines = ctx.textFile(args[0], 1); // Loads all URLs from input file and initialize their neighbors. JavaPairRDD> links = lines.mapToPair(new PairFunction() { @@ -87,7 +90,7 @@ public Double call(Iterable rs) { }); // Calculates and updates URL ranks continuously using PageRank algorithm. - for (int current = 0; current < Integer.parseInt(args[2]); current++) { + for (int current = 0; current < Integer.parseInt(args[1]); current++) { // Calculates URL contributions to the rank of other URLs. JavaPairRDD contribs = links.join(ranks).values() .flatMapToPair(new PairFlatMapFunction, Double>, String, Double>() { diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java index ac8df02c4630b..11157d7573fae 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java @@ -17,6 +17,7 @@ package org.apache.spark.examples; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; @@ -25,19 +26,18 @@ import java.util.ArrayList; import java.util.List; -/** Computes an approximation to pi */ +/** + * Computes an approximation to pi + * Usage: JavaSparkPi [slices] + */ public final class JavaSparkPi { + public static void main(String[] args) throws Exception { - if (args.length == 0) { - System.err.println("Usage: JavaSparkPi [slices]"); - System.exit(1); - } - - JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaSparkPi", - System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaSparkPi.class)); + SparkConf sparkConf = new SparkConf().setAppName("JavaSparkPi"); + JavaSparkContext jsc = new JavaSparkContext(sparkConf); - int slices = (args.length == 2) ? Integer.parseInt(args[1]) : 2; + int slices = (args.length == 1) ? Integer.parseInt(args[0]) : 2; int n = 100000 * slices; List l = new ArrayList(n); for (int i = 0; i < n; i++) { diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index d66b9ba265fe8..2563fcdd234bb 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -17,19 +17,22 @@ package org.apache.spark.examples; -import scala.Tuple2; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.PairFunction; - import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Random; import java.util.Set; +import scala.Tuple2; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.PairFunction; + /** * Transitive closure on a graph, implemented in Java. + * Usage: JavaTC [slices] */ public final class JavaTC { @@ -61,14 +64,9 @@ public Tuple2 call(Tuple2> t } public static void main(String[] args) { - if (args.length == 0) { - System.err.println("Usage: JavaTC []"); - System.exit(1); - } - - JavaSparkContext sc = new JavaSparkContext(args[0], "JavaTC", - System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaTC.class)); - Integer slices = (args.length > 1) ? Integer.parseInt(args[1]): 2; + SparkConf sparkConf = new SparkConf().setAppName("JavaHdfsLR"); + JavaSparkContext sc = new JavaSparkContext(sparkConf); + Integer slices = (args.length > 0) ? Integer.parseInt(args[0]): 2; JavaPairRDD tc = sc.parallelizePairs(generateGraph(), slices).cache(); // Linear transitive closure: each round grows paths by one edge, 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 87c1b80981961..9a6a944f7edef 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -18,6 +18,7 @@ package org.apache.spark.examples; import scala.Tuple2; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -33,14 +34,15 @@ public final class JavaWordCount { private static final Pattern SPACE = Pattern.compile(" "); public static void main(String[] args) throws Exception { - if (args.length < 2) { - System.err.println("Usage: JavaWordCount "); + + if (args.length < 1) { + System.err.println("Usage: JavaWordCount "); System.exit(1); } - JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaWordCount", - System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaWordCount.class)); - JavaRDD lines = ctx.textFile(args[1], 1); + SparkConf sparkConf = new SparkConf().setAppName("JavaWordCount"); + JavaSparkContext ctx = new JavaSparkContext(sparkConf); + JavaRDD lines = ctx.textFile(args[0], 1); JavaRDD words = lines.flatMap(new FlatMapFunction() { @Override diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java index 4533c4c5f241a..8d381d4e0a943 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java @@ -17,6 +17,7 @@ package org.apache.spark.examples.mllib; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; @@ -57,23 +58,22 @@ public String call(Tuple2 element) { public static void main(String[] args) { - if (args.length != 5 && args.length != 6) { + if (args.length < 4) { System.err.println( - "Usage: JavaALS []"); + "Usage: JavaALS []"); System.exit(1); } - - int rank = Integer.parseInt(args[2]); - int iterations = Integer.parseInt(args[3]); - String outputDir = args[4]; + SparkConf sparkConf = new SparkConf().setAppName("JavaALS"); + int rank = Integer.parseInt(args[1]); + int iterations = Integer.parseInt(args[2]); + String outputDir = args[3]; int blocks = -1; - if (args.length == 6) { - blocks = Integer.parseInt(args[5]); + if (args.length == 5) { + blocks = Integer.parseInt(args[4]); } - JavaSparkContext sc = new JavaSparkContext(args[0], "JavaALS", - System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaALS.class)); - JavaRDD lines = sc.textFile(args[1]); + JavaSparkContext sc = new JavaSparkContext(sparkConf); + JavaRDD lines = sc.textFile(args[0]); JavaRDD ratings = lines.map(new ParseRating()); diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java index 0cfb8e69ed28f..f796123a25727 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java @@ -19,6 +19,7 @@ import java.util.regex.Pattern; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; @@ -48,24 +49,21 @@ public Vector call(String line) { } public static void main(String[] args) { - - if (args.length < 4) { + if (args.length < 3) { System.err.println( - "Usage: JavaKMeans []"); + "Usage: JavaKMeans []"); System.exit(1); } - - String inputFile = args[1]; - int k = Integer.parseInt(args[2]); - int iterations = Integer.parseInt(args[3]); + String inputFile = args[0]; + int k = Integer.parseInt(args[1]); + int iterations = Integer.parseInt(args[2]); int runs = 1; - if (args.length >= 5) { - runs = Integer.parseInt(args[4]); + if (args.length >= 4) { + runs = Integer.parseInt(args[3]); } - - JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans", - System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaKMeans.class)); + SparkConf sparkConf = new SparkConf().setAppName("JavaKMeans"); + JavaSparkContext sc = new JavaSparkContext(sparkConf); JavaRDD lines = sc.textFile(inputFile); JavaRDD points = lines.map(new ParsePoint()); diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLR.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLR.java index f6e48b498727b..eceb6927d5551 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLR.java @@ -19,6 +19,7 @@ import java.util.regex.Pattern; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; @@ -51,17 +52,16 @@ public LabeledPoint call(String line) { } public static void main(String[] args) { - if (args.length != 4) { - System.err.println("Usage: JavaLR "); + if (args.length != 3) { + System.err.println("Usage: JavaLR "); System.exit(1); } - - JavaSparkContext sc = new JavaSparkContext(args[0], "JavaLR", - System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaLR.class)); - JavaRDD lines = sc.textFile(args[1]); + SparkConf sparkConf = new SparkConf().setAppName("JavaLR"); + JavaSparkContext sc = new JavaSparkContext(sparkConf); + JavaRDD lines = sc.textFile(args[0]); JavaRDD points = lines.map(new ParsePoint()).cache(); - double stepSize = Double.parseDouble(args[2]); - int iterations = Integer.parseInt(args[3]); + double stepSize = Double.parseDouble(args[1]); + int iterations = Integer.parseInt(args[2]); // Another way to configure LogisticRegression // @@ -73,7 +73,7 @@ public static void main(String[] args) { // LogisticRegressionModel model = lr.train(points.rdd()); LogisticRegressionModel model = LogisticRegressionWithSGD.train(points.rdd(), - iterations, stepSize); + iterations, stepSize); System.out.print("Final w: " + model.weights()); diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index d62a72f53443c..ad5ec84b71e69 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.util.List; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; @@ -51,8 +52,8 @@ public void setAge(int age) { } public static void main(String[] args) throws Exception { - JavaSparkContext ctx = new JavaSparkContext("local", "JavaSparkSQL", - System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaSparkSQL.class)); + SparkConf sparkConf = new SparkConf().setAppName("JavaSparkSQL"); + JavaSparkContext ctx = new JavaSparkContext(sparkConf); JavaSQLContext sqlCtx = new JavaSQLContext(ctx); // Load a text file and convert each line to a Java Bean. diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java index a5ece68cef870..400b68c2215b3 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java @@ -17,6 +17,7 @@ package org.apache.spark.examples.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.Function; import org.apache.spark.examples.streaming.StreamingExamples; import org.apache.spark.streaming.*; @@ -31,9 +32,8 @@ * an Avro server on at the request host:port address and listen for requests. * Your Flume AvroSink should be pointed to this address. * - * Usage: JavaFlumeEventCount + * Usage: JavaFlumeEventCount * - * is a Spark master URL * is the host the Flume receiver will be started on - a receiver * creates a server and listens for flume events. * is the port the Flume receiver will listen on. @@ -43,22 +43,19 @@ private JavaFlumeEventCount() { } public static void main(String[] args) { - if (args.length != 3) { - System.err.println("Usage: JavaFlumeEventCount "); + if (args.length != 2) { + System.err.println("Usage: JavaFlumeEventCount "); System.exit(1); } StreamingExamples.setStreamingLogLevels(); - String master = args[0]; - String host = args[1]; - int port = Integer.parseInt(args[2]); + String host = args[0]; + int port = Integer.parseInt(args[1]); Duration batchInterval = new Duration(2000); - - JavaStreamingContext ssc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval, - System.getenv("SPARK_HOME"), - JavaStreamingContext.jarOfClass(JavaFlumeEventCount.class)); + SparkConf sparkConf = new SparkConf().setAppName("JavaFlumeEventCount"); + JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, batchInterval); JavaReceiverInputDStream flumeStream = FlumeUtils.createStream(ssc, "localhost", port); flumeStream.count(); diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java index da51eb189a649..6a74cc50d19ed 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java @@ -21,7 +21,11 @@ import java.util.HashMap; import java.util.regex.Pattern; + +import scala.Tuple2; + import com.google.common.collect.Lists; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.Function2; @@ -33,19 +37,18 @@ import org.apache.spark.streaming.api.java.JavaPairReceiverInputDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.apache.spark.streaming.kafka.KafkaUtils; -import scala.Tuple2; /** * Consumes messages from one or more topics in Kafka and does wordcount. - * Usage: JavaKafkaWordCount - * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * Usage: JavaKafkaWordCount * is a list of one or more zookeeper servers that make quorum * is the name of kafka consumer group * is a list of one or more kafka topics to consume from * is the number of threads the kafka consumer should use * * Example: - * `./bin/run-example org.apache.spark.examples.streaming.JavaKafkaWordCount local[2] zoo01,zoo02, + * `./bin/spark-submit examples.jar \ + * --class org.apache.spark.examples.streaming.JavaKafkaWordCount zoo01,zoo02, \ * zoo03 my-consumer-group topic1,topic2 1` */ @@ -56,27 +59,25 @@ private JavaKafkaWordCount() { } public static void main(String[] args) { - if (args.length < 5) { - System.err.println("Usage: KafkaWordCount "); + if (args.length < 4) { + System.err.println("Usage: JavaKafkaWordCount "); System.exit(1); } StreamingExamples.setStreamingLogLevels(); - + SparkConf sparkConf = new SparkConf().setAppName("JavaKafkaWordCount"); // Create the context with a 1 second batch size - JavaStreamingContext jssc = new JavaStreamingContext(args[0], "KafkaWordCount", - new Duration(2000), System.getenv("SPARK_HOME"), - JavaStreamingContext.jarOfClass(JavaKafkaWordCount.class)); + JavaStreamingContext jssc = new JavaStreamingContext(sparkConf, new Duration(2000)); - int numThreads = Integer.parseInt(args[4]); + int numThreads = Integer.parseInt(args[3]); Map topicMap = new HashMap(); - String[] topics = args[3].split(","); + String[] topics = args[2].split(","); for (String topic: topics) { topicMap.put(topic, numThreads); } JavaPairReceiverInputDStream messages = - KafkaUtils.createStream(jssc, args[1], args[2], topicMap); + KafkaUtils.createStream(jssc, args[0], args[1], topicMap); JavaDStream lines = messages.map(new Function, String>() { @Override diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java index ac84991d87b8b..e5cbd39f437c2 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java @@ -17,9 +17,10 @@ package org.apache.spark.examples.streaming; -import com.google.common.collect.Lists; -import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import scala.Tuple2; +import com.google.common.collect.Lists; + +import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; @@ -27,41 +28,39 @@ import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; import java.util.regex.Pattern; /** * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. - * Usage: JavaNetworkWordCount - * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * Usage: JavaNetworkWordCount * and describe the TCP server that Spark Streaming would connect to receive data. * * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run org.apache.spark.examples.streaming.JavaNetworkWordCount local[2] localhost 9999` + * `$ ./bin/spark-submit examples.jar \ + * --class org.apache.spark.examples.streaming.JavaNetworkWordCount localhost 9999` */ public final class JavaNetworkWordCount { private static final Pattern SPACE = Pattern.compile(" "); public static void main(String[] args) { - if (args.length < 3) { - System.err.println("Usage: JavaNetworkWordCount \n" + - "In local mode, should be 'local[n]' with n > 1"); + if (args.length < 2) { + System.err.println("Usage: JavaNetworkWordCount "); System.exit(1); } StreamingExamples.setStreamingLogLevels(); - + SparkConf sparkConf = new SparkConf().setAppName("JavaNetworkWordCount"); // Create the context with a 1 second batch size - JavaStreamingContext ssc = new JavaStreamingContext(args[0], "JavaNetworkWordCount", - new Duration(1000), System.getenv("SPARK_HOME"), - JavaStreamingContext.jarOfClass(JavaNetworkWordCount.class)); + JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, new Duration(1000)); // Create a JavaReceiverInputDStream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') - JavaReceiverInputDStream lines = ssc.socketTextStream(args[1], Integer.parseInt(args[2])); + JavaReceiverInputDStream lines = ssc.socketTextStream(args[0], Integer.parseInt(args[1])); JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java index 819311968fac5..4ce8437f82705 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java @@ -17,8 +17,16 @@ package org.apache.spark.examples.streaming; -import com.google.common.collect.Lists; + +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; + import scala.Tuple2; + +import com.google.common.collect.Lists; + +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; @@ -28,25 +36,17 @@ import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; -import java.util.LinkedList; -import java.util.List; -import java.util.Queue; - public final class JavaQueueStream { private JavaQueueStream() { } public static void main(String[] args) throws Exception { - if (args.length < 1) { - System.err.println("Usage: JavaQueueStream "); - System.exit(1); - } StreamingExamples.setStreamingLogLevels(); + SparkConf sparkConf = new SparkConf().setAppName("JavaQueueStream"); // Create the context - JavaStreamingContext ssc = new JavaStreamingContext(args[0], "QueueStream", new Duration(1000), - System.getenv("SPARK_HOME"), JavaStreamingContext.jarOfClass(JavaQueueStream.class)); + JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, new Duration(1000)); // Create the queue through which RDDs can be pushed to // a QueueInputDStream diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index f6dfd2c4c6217..973049b95a7bd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -17,28 +17,26 @@ package org.apache.spark.examples -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} +/** + * Usage: BroadcastTest [slices] [numElem] [broadcastAlgo] [blockSize] + */ object BroadcastTest { def main(args: Array[String]) { - if (args.length == 0) { - System.err.println("Usage: BroadcastTest [slices] [numElem] [broadcastAlgo]" + - " [blockSize]") - System.exit(1) - } - val bcName = if (args.length > 3) args(3) else "Http" - val blockSize = if (args.length > 4) args(4) else "4096" + val bcName = if (args.length > 2) args(2) else "Http" + val blockSize = if (args.length > 3) args(3) else "4096" System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast." + bcName + "BroadcastFactory") System.setProperty("spark.broadcast.blockSize", blockSize) + val sparkConf = new SparkConf().setAppName("Broadcast Test") - val sc = new SparkContext(args(0), "Broadcast Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) + val sc = new SparkContext(sparkConf) - val slices = if (args.length > 1) args(1).toInt else 2 - val num = if (args.length > 2) args(2).toInt else 1000000 + val slices = if (args.length > 0) args(0).toInt else 2 + val num = if (args.length > 1) args(1).toInt else 1000000 val arr1 = new Array[Int](num) for (i <- 0 until arr1.length) { diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala index 3798329fc2f41..9a00701f985f0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala @@ -30,7 +30,7 @@ import org.apache.cassandra.hadoop.cql3.CqlOutputFormat import org.apache.cassandra.utils.ByteBufferUtil import org.apache.hadoop.mapreduce.Job -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ /* @@ -65,19 +65,18 @@ import org.apache.spark.SparkContext._ /** * This example demonstrates how to read and write to cassandra column family created using CQL3 * using Spark. - * Parameters : - * Usage: ./bin/run-example org.apache.spark.examples.CassandraCQLTest local[2] localhost 9160 - * + * Parameters : + * Usage: ./bin/spark-submit examples.jar \ + * --class org.apache.spark.examples.CassandraCQLTest localhost 9160 */ object CassandraCQLTest { def main(args: Array[String]) { - val sc = new SparkContext(args(0), - "CQLTestApp", - System.getenv("SPARK_HOME"), - SparkContext.jarOfClass(this.getClass).toSeq) - val cHost: String = args(1) - val cPort: String = args(2) + val sparkConf = new SparkConf().setAppName("CQLTestApp") + + val sc = new SparkContext(sparkConf) + val cHost: String = args(0) + val cPort: String = args(1) val KeySpace = "retail" val InputColumnFamily = "ordercf" val OutputColumnFamily = "salecount" diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala index ed5d2f9e46f29..91ba364a346a5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala @@ -30,7 +30,7 @@ import org.apache.cassandra.thrift._ import org.apache.cassandra.utils.ByteBufferUtil import org.apache.hadoop.mapreduce.Job -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ /* @@ -38,10 +38,10 @@ import org.apache.spark.SparkContext._ * support for Hadoop. * * To run this example, run this file with the following command params - - * + * * * So if you want to run this on localhost this will be, - * local[3] localhost 9160 + * localhost 9160 * * The example makes some assumptions: * 1. You have already created a keyspace called casDemo and it has a column family named Words @@ -54,9 +54,9 @@ import org.apache.spark.SparkContext._ object CassandraTest { def main(args: Array[String]) { - + val sparkConf = new SparkConf().setAppName("casDemo") // Get a SparkContext - val sc = new SparkContext(args(0), "casDemo") + val sc = new SparkContext(sparkConf) // Build the job configuration with ConfigHelper provided by Cassandra val job = new Job() diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala index f0dcef431b2e1..d42f63e87052e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala @@ -17,17 +17,12 @@ package org.apache.spark.examples -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} object ExceptionHandlingTest { def main(args: Array[String]) { - if (args.length == 0) { - System.err.println("Usage: ExceptionHandlingTest ") - System.exit(1) - } - - val sc = new SparkContext(args(0), "ExceptionHandlingTest", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) + val sparkConf = new SparkConf().setAppName("ExceptionHandlingTest") + val sc = new SparkContext(sparkConf) sc.parallelize(0 until sc.defaultParallelism).foreach { i => if (math.random > 0.75) { throw new Exception("Testing exception handling") diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index e67bb29a49405..efd91bb054981 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -19,24 +19,21 @@ package org.apache.spark.examples import java.util.Random -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ +/** + * Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers] + */ object GroupByTest { def main(args: Array[String]) { - if (args.length == 0) { - System.err.println( - "Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers]") - System.exit(1) - } - - var numMappers = if (args.length > 1) args(1).toInt else 2 - var numKVPairs = if (args.length > 2) args(2).toInt else 1000 - var valSize = if (args.length > 3) args(3).toInt else 1000 - var numReducers = if (args.length > 4) args(4).toInt else numMappers - - val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) + val sparkConf = new SparkConf().setAppName("GroupBy Test") + var numMappers = if (args.length > 0) args(0).toInt else 2 + var numKVPairs = if (args.length > 1) args(1).toInt else 1000 + var valSize = if (args.length > 2) args(2).toInt else 1000 + var numReducers = if (args.length > 3) args(3).toInt else numMappers + + val sc = new SparkContext(sparkConf) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala index adbd1c02fa2ea..a8c338480e6e2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala @@ -26,11 +26,9 @@ import org.apache.spark.rdd.NewHadoopRDD object HBaseTest { def main(args: Array[String]) { - val sc = new SparkContext(args(0), "HBaseTest", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) - + val sparkConf = new SparkConf().setAppName("HBaseTest") + val sc = new SparkContext(sparkConf) val conf = HBaseConfiguration.create() - // Other options for configuring scan behavior are available. More information available at // http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html conf.set(TableInputFormat.INPUT_TABLE, args(1)) diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index c7a4884af10b7..331de3ad1ef53 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -21,8 +21,8 @@ import org.apache.spark._ object HdfsTest { def main(args: Array[String]) { - val sc = new SparkContext(args(0), "HdfsTest", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) + val sparkConf = new SparkConf().setAppName("HdfsTest") + val sc = new SparkContext(sparkConf) val file = sc.textFile(args(1)) val mapped = file.map(s => s.length).cache() for (iter <- 1 to 10) { diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index f77a444ff7a9f..4c655b84fde2e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -17,11 +17,13 @@ package org.apache.spark.examples -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ /** * Executes a roll up-style query against Apache logs. + * + * Usage: LogQuery [logFile] */ object LogQuery { val exampleApacheLogs = List( @@ -40,16 +42,12 @@ object LogQuery { ) def main(args: Array[String]) { - if (args.length == 0) { - System.err.println("Usage: LogQuery [logFile]") - System.exit(1) - } - val sc = new SparkContext(args(0), "Log Query", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) + val sparkConf = new SparkConf().setAppName("Log Query") + val sc = new SparkContext(sparkConf) val dataSet = - if (args.length == 2) sc.textFile(args(1)) else sc.parallelize(exampleApacheLogs) + if (args.length == 1) sc.textFile(args(0)) else sc.parallelize(exampleApacheLogs) // scalastyle:off val apacheLogRegex = """^([\d.]+) (\S+) (\S+) \[([\w\d:/]+\s[+\-]\d{4})\] "(.+?)" (\d{3}) ([\d\-]+) "([^"]+)" "([^"]+)".*""".r diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index c8985eae33de3..2a5c0c0defe13 100644 --- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -18,20 +18,19 @@ package org.apache.spark.examples import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} +/** + * Usage: MultiBroadcastTest [slices] [numElem] + */ object MultiBroadcastTest { def main(args: Array[String]) { - if (args.length == 0) { - System.err.println("Usage: MultiBroadcastTest [] [numElem]") - System.exit(1) - } - val sc = new SparkContext(args(0), "Multi-Broadcast Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) + val sparkConf = new SparkConf().setAppName("Multi-Broadcast Test") + val sc = new SparkContext(sparkConf) - val slices = if (args.length > 1) args(1).toInt else 2 - val num = if (args.length > 2) args(2).toInt else 1000000 + val slices = if (args.length > 0) args(0).toInt else 2 + val num = if (args.length > 1) args(1).toInt else 1000000 val arr1 = new Array[Int](num) for (i <- 0 until arr1.length) { diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 54e8503711e30..5291ab81f459e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -19,25 +19,23 @@ package org.apache.spark.examples import java.util.Random -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ +/** + * Usage: SimpleSkewedGroupByTest [numMappers] [numKVPairs] [valSize] [numReducers] [ratio] + */ object SimpleSkewedGroupByTest { def main(args: Array[String]) { - if (args.length == 0) { - System.err.println("Usage: SimpleSkewedGroupByTest " + - "[numMappers] [numKVPairs] [valSize] [numReducers] [ratio]") - System.exit(1) - } - var numMappers = if (args.length > 1) args(1).toInt else 2 - var numKVPairs = if (args.length > 2) args(2).toInt else 1000 - var valSize = if (args.length > 3) args(3).toInt else 1000 - var numReducers = if (args.length > 4) args(4).toInt else numMappers - var ratio = if (args.length > 5) args(5).toInt else 5.0 + val sparkConf = new SparkConf().setAppName("SimpleSkewedGroupByTest") + var numMappers = if (args.length > 0) args(0).toInt else 2 + var numKVPairs = if (args.length > 1) args(1).toInt else 1000 + var valSize = if (args.length > 2) args(2).toInt else 1000 + var numReducers = if (args.length > 3) args(3).toInt else numMappers + var ratio = if (args.length > 4) args(4).toInt else 5.0 - val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) + val sc = new SparkContext(sparkConf) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 1c5f22e1c00bb..017d4e1e5ce13 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -19,24 +19,21 @@ package org.apache.spark.examples import java.util.Random -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ +/** + * Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers] + */ object SkewedGroupByTest { def main(args: Array[String]) { - if (args.length == 0) { - System.err.println( - "Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers]") - System.exit(1) - } - - var numMappers = if (args.length > 1) args(1).toInt else 2 - var numKVPairs = if (args.length > 2) args(2).toInt else 1000 - var valSize = if (args.length > 3) args(3).toInt else 1000 - var numReducers = if (args.length > 4) args(4).toInt else numMappers - - val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) + val sparkConf = new SparkConf().setAppName("GroupBy Test") + var numMappers = if (args.length > 0) args(0).toInt else 2 + var numKVPairs = if (args.length > 1) args(1).toInt else 1000 + var valSize = if (args.length > 2) args(2).toInt else 1000 + var numReducers = if (args.length > 3) args(3).toInt else numMappers + + val sc = new SparkContext(sparkConf) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random 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 0dc726aecdd28..5cbc966bf06ca 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -88,32 +88,24 @@ object SparkALS { } def main(args: Array[String]) { - if (args.length == 0) { - System.err.println("Usage: SparkALS [ ]") - System.exit(1) - } - - var host = "" var slices = 0 - val options = (0 to 5).map(i => if (i < args.length) Some(args(i)) else None) + val options = (0 to 4).map(i => if (i < args.length) Some(args(i)) else None) options.toArray match { - case Array(host_, m, u, f, iters, slices_) => - host = host_.get + case Array(m, u, f, iters, slices_) => M = m.getOrElse("100").toInt U = u.getOrElse("500").toInt F = f.getOrElse("10").toInt ITERATIONS = iters.getOrElse("5").toInt slices = slices_.getOrElse("2").toInt case _ => - System.err.println("Usage: SparkALS [ ]") + System.err.println("Usage: SparkALS [M] [U] [F] [iters] [slices]") System.exit(1) } printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) - - val sc = new SparkContext(host, "SparkALS", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) + val sparkConf = new SparkConf().setAppName("SparkALS") + val sc = new SparkContext(sparkConf) val R = generateR() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 3a6f18c33ea4b..4906a696e90a7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -49,20 +49,21 @@ object SparkHdfsLR { } def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: SparkHdfsLR ") + if (args.length < 2) { + System.err.println("Usage: SparkHdfsLR ") System.exit(1) } - val inputPath = args(1) + + val sparkConf = new SparkConf().setAppName("SparkHdfsLR") + val inputPath = args(0) val conf = SparkHadoopUtil.get.newConfiguration() - val sc = new SparkContext(args(0), "SparkHdfsLR", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq, Map(), + val sc = new SparkContext(sparkConf, InputFormatInfo.computePreferredLocations( Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) )) val lines = sc.textFile(inputPath) val points = lines.map(parsePoint _).cache() - val ITERATIONS = args(2).toInt + val ITERATIONS = args(1).toInt // Initialize w to a random value var w = DenseVector.fill(D){2 * rand.nextDouble - 1} diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index dcae9591b0407..4d28e0aad6597 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -21,7 +21,7 @@ import java.util.Random import breeze.linalg.{Vector, DenseVector, squaredDistance} -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ /** @@ -52,16 +52,16 @@ object SparkKMeans { } def main(args: Array[String]) { - if (args.length < 4) { - System.err.println("Usage: SparkLocalKMeans ") - System.exit(1) + if (args.length < 3) { + System.err.println("Usage: SparkKMeans ") + System.exit(1) } - val sc = new SparkContext(args(0), "SparkLocalKMeans", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) - val lines = sc.textFile(args(1)) + val sparkConf = new SparkConf().setAppName("SparkKMeans") + val sc = new SparkContext(sparkConf) + val lines = sc.textFile(args(0)) val data = lines.map(parseVector _).cache() - val K = args(2).toInt - val convergeDist = args(3).toDouble + val K = args(1).toInt + val convergeDist = args(2).toDouble val kPoints = data.takeSample(withReplacement = false, K, 42).toArray var tempDist = 1.0 diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index 4f74882ccbea5..99ceb3089e9fe 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -27,6 +27,7 @@ import org.apache.spark._ /** * Logistic regression based classification. + * Usage: SparkLR [slices] */ object SparkLR { val N = 10000 // Number of data points @@ -47,13 +48,9 @@ object SparkLR { } def main(args: Array[String]) { - if (args.length == 0) { - System.err.println("Usage: SparkLR []") - System.exit(1) - } - val sc = new SparkContext(args(0), "SparkLR", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) - val numSlices = if (args.length > 1) args(1).toInt else 2 + val sparkConf = new SparkConf().setAppName("SparkLR") + val sc = new SparkContext(sparkConf) + val numSlices = if (args.length > 0) args(0).toInt else 2 val points = sc.parallelize(generateData, numSlices).cache() // Initialize w to a random value diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index fa41c5c560943..40b36c779afd6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -18,7 +18,7 @@ package org.apache.spark.examples import org.apache.spark.SparkContext._ -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} /** * Computes the PageRank of URLs from an input file. Input file should @@ -31,14 +31,10 @@ import org.apache.spark.SparkContext */ object SparkPageRank { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: PageRank ") - System.exit(1) - } - var iters = args(2).toInt - val ctx = new SparkContext(args(0), "PageRank", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) - val lines = ctx.textFile(args(1), 1) + val sparkConf = new SparkConf().setAppName("PageRank") + var iters = args(1).toInt + val ctx = new SparkContext(sparkConf) + val lines = ctx.textFile(args(0), 1) val links = lines.map{ s => val parts = s.split("\\s+") (parts(0), parts(1)) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index d8f5720504223..9fbb0a800d735 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -24,13 +24,9 @@ import org.apache.spark._ /** Computes an approximation to pi */ object SparkPi { def main(args: Array[String]) { - if (args.length == 0) { - System.err.println("Usage: SparkPi []") - System.exit(1) - } - val spark = new SparkContext(args(0), "SparkPi", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) - val slices = if (args.length > 1) args(1).toInt else 2 + val conf = new SparkConf().setAppName("Spark Pi") + val spark = new SparkContext(conf) + val slices = if (args.length > 0) args(0).toInt else 2 val n = 100000 * slices val count = spark.parallelize(1 to n, slices).map { i => val x = random * 2 - 1 diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index 17d983cd875db..f7f83086df3db 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples import scala.util.Random import scala.collection.mutable -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ /** @@ -42,13 +42,9 @@ object SparkTC { } def main(args: Array[String]) { - if (args.length == 0) { - System.err.println("Usage: SparkTC []") - System.exit(1) - } - val spark = new SparkContext(args(0), "SparkTC", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) - val slices = if (args.length > 1) args(1).toInt else 2 + val sparkConf = new SparkConf().setAppName("SparkTC") + val spark = new SparkContext(sparkConf) + val slices = if (args.length > 0) args(0).toInt else 2 var tc = spark.parallelize(generateGraph, slices).cache() // Linear transitive closure: each round grows paths by one edge, diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala index 7e43c384bdb9d..22127621867e1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala @@ -51,20 +51,16 @@ object SparkTachyonHdfsLR { } def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: SparkTachyonHdfsLR ") - System.exit(1) - } - val inputPath = args(1) + val inputPath = args(0) val conf = SparkHadoopUtil.get.newConfiguration() - val sc = new SparkContext(args(0), "SparkTachyonHdfsLR", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq, Map(), + val sparkConf = new SparkConf().setAppName("SparkTachyonHdfsLR") + val sc = new SparkContext(sparkConf, InputFormatInfo.computePreferredLocations( Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) )) val lines = sc.textFile(inputPath) val points = lines.map(parsePoint _).persist(StorageLevel.OFF_HEAP) - val ITERATIONS = args(2).toInt + val ITERATIONS = args(1).toInt // Initialize w to a random value var w = DenseVector.fill(D){2 * rand.nextDouble - 1} diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala index 93459110e4e0e..7743f7968b100 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala @@ -28,14 +28,10 @@ import org.apache.spark.storage.StorageLevel */ object SparkTachyonPi { def main(args: Array[String]) { - if (args.length == 0) { - System.err.println("Usage: SparkTachyonPi []") - System.exit(1) - } - val spark = new SparkContext(args(0), "SparkTachyonPi", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) + val sparkConf = new SparkConf().setAppName("SparkTachyonPi") + val spark = new SparkContext(sparkConf) - val slices = if (args.length > 1) args(1).toInt else 2 + val slices = if (args.length > 0) args(0).toInt else 2 val n = 100000 * slices val rdd = spark.parallelize(1 to n, slices) diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala index 25bd55ca88b94..235c3bf820244 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala @@ -32,22 +32,22 @@ import scala.xml.{XML,NodeSeq} */ object WikipediaPageRank { def main(args: Array[String]) { - if (args.length < 5) { + if (args.length < 4) { System.err.println( - "Usage: WikipediaPageRank ") + "Usage: WikipediaPageRank ") System.exit(-1) } val sparkConf = new SparkConf() + sparkConf.setAppName("WikipediaPageRank") sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") sparkConf.set("spark.kryo.registrator", classOf[PRKryoRegistrator].getName) val inputFile = args(0) val threshold = args(1).toDouble val numPartitions = args(2).toInt - val host = args(3) - val usePartitioner = args(4).toBoolean + val usePartitioner = args(3).toBoolean - sparkConf.setMaster(host).setAppName("WikipediaPageRank") + sparkConf.setAppName("WikipediaPageRank") val sc = new SparkContext(sparkConf) // Parse the Wikipedia page data into a graph diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index dee3cb6c0abae..a197dac87d6db 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -30,22 +30,20 @@ import org.apache.spark.rdd.RDD object WikipediaPageRankStandalone { def main(args: Array[String]) { - if (args.length < 5) { + if (args.length < 4) { System.err.println("Usage: WikipediaPageRankStandalone " + - " ") + " ") System.exit(-1) } val sparkConf = new SparkConf() sparkConf.set("spark.serializer", "spark.bagel.examples.WPRSerializer") - val inputFile = args(0) val threshold = args(1).toDouble val numIterations = args(2).toInt - val host = args(3) - val usePartitioner = args(4).toBoolean + val usePartitioner = args(3).toBoolean - sparkConf.setMaster(host).setAppName("WikipediaPageRankStandalone") + sparkConf.setAppName("WikipediaPageRankStandalone") val sc = new SparkContext(sparkConf) diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala index d58fddff2b5ec..6ef3b62dcbedc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala @@ -28,9 +28,9 @@ import org.apache.spark.graphx.lib.Analytics */ object LiveJournalPageRank { def main(args: Array[String]) { - if (args.length < 2) { + if (args.length < 1) { System.err.println( - "Usage: LiveJournalPageRank \n" + + "Usage: LiveJournalPageRank \n" + " [--tol=]\n" + " The tolerance allowed at convergence (smaller => more accurate). Default is " + "0.001.\n" + @@ -44,6 +44,6 @@ object LiveJournalPageRank { System.exit(-1) } - Analytics.main(args.patch(1, List("pagerank"), 0)) + Analytics.main(args.patch(0, List("pagerank"), 0)) } } diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index ff9254b044c24..61c460c6b1de8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -17,7 +17,7 @@ package org.apache.spark.examples.sql -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql.SQLContext // One method for defining the schema of an RDD is to make a case class with the desired column @@ -26,7 +26,8 @@ case class Record(key: Int, value: String) object RDDRelation { def main(args: Array[String]) { - val sc = new SparkContext("local", "RDDRelation") + val sparkConf = new SparkConf().setAppName("RDDRelation") + val sc = new SparkContext(sparkConf) val sqlContext = new SQLContext(sc) // Importing the SQL context gives access to all the SQL functions and implicit conversions. diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 66ce93a26ef42..b262fabbe0e0d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -17,7 +17,7 @@ package org.apache.spark.examples.sql.hive -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql._ import org.apache.spark.sql.hive.LocalHiveContext @@ -25,7 +25,8 @@ object HiveFromSpark { case class Record(key: Int, value: String) def main(args: Array[String]) { - val sc = new SparkContext("local", "HiveFromSpark") + val sparkConf = new SparkConf().setAppName("HiveFromSpark") + val sc = new SparkContext(sparkConf) // A local hive context creates an instance of the Hive Metastore in process, storing the // the warehouse data in the current directory. This location can be overridden by diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala index 84cf43df0f96c..e29e16a9c1b17 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala @@ -126,31 +126,30 @@ object FeederActor { /** * A sample word count program demonstrating the use of plugging in * Actor as Receiver - * Usage: ActorWordCount - * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * Usage: ActorWordCount * and describe the AkkaSystem that Spark Sample feeder is running on. * * To run this example locally, you may run Feeder Actor as - * `$ ./bin/run-example org.apache.spark.examples.streaming.FeederActor 127.0.1.1 9999` + * `./bin/spark-submit examples.jar \ + * --class org.apache.spark.examples.streaming.FeederActor 127.0.1.1 9999` * and then run the example - * `./bin/run-example org.apache.spark.examples.streaming.ActorWordCount local[2] 127.0.1.1 9999` + * `./bin/spark-submit examples.jar --class org.apache.spark.examples.streaming.ActorWordCount \ + * 127.0.1.1 9999` */ object ActorWordCount { def main(args: Array[String]) { - if (args.length < 3) { + if (args.length < 2) { System.err.println( - "Usage: ActorWordCount " + - "In local mode, should be 'local[n]' with n > 1") + "Usage: ActorWordCount ") System.exit(1) } StreamingExamples.setStreamingLogLevels() - val Seq(master, host, port) = args.toSeq - + val Seq(host, port) = args.toSeq + val sparkConf = new SparkConf().setAppName("ActorWordCount") // Create the context and set the batch size - val ssc = new StreamingContext(master, "ActorWordCount", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + val ssc = new StreamingContext(sparkConf, Seconds(2)) /* * Following is the use of actorStream to plug in custom actor as receiver diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala index 5b2a1035fc779..38362edac27f8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala @@ -17,6 +17,7 @@ package org.apache.spark.examples.streaming +import org.apache.spark.SparkConf import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.streaming.flume._ @@ -29,9 +30,8 @@ import org.apache.spark.util.IntParam * an Avro server on at the request host:port address and listen for requests. * Your Flume AvroSink should be pointed to this address. * - * Usage: FlumeEventCount + * Usage: FlumeEventCount * - * is a Spark master URL * is the host the Flume receiver will be started on - a receiver * creates a server and listens for flume events. * is the port the Flume receiver will listen on. @@ -40,21 +40,21 @@ object FlumeEventCount { def main(args: Array[String]) { if (args.length != 3) { System.err.println( - "Usage: FlumeEventCount ") + "Usage: FlumeEventCount ") System.exit(1) } StreamingExamples.setStreamingLogLevels() - val Array(master, host, IntParam(port)) = args + val Array(host, IntParam(port)) = args val batchInterval = Milliseconds(2000) + val sparkConf = new SparkConf().setAppName("FlumeEventCount") // Create the context and set the batch size - val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval, - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + val ssc = new StreamingContext(sparkConf, batchInterval) // Create a flume stream - val stream = FlumeUtils.createStream(ssc, host,port,StorageLevel.MEMORY_ONLY_SER_2) + val stream = FlumeUtils.createStream(ssc, host, port, StorageLevel.MEMORY_ONLY_SER_2) // Print out the count of events received from this server in each batch stream.count().map(cnt => "Received " + cnt + " flume events." ).print() diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala index b440956ba3137..55ac48cfb6d10 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala @@ -17,35 +17,35 @@ package org.apache.spark.examples.streaming +import org.apache.spark.SparkConf import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ /** * Counts words in new text files created in the given directory - * Usage: HdfsWordCount - * is the Spark master URL. + * Usage: HdfsWordCount * is the directory that Spark Streaming will use to find and read new text files. * * To run this on your local machine on directory `localdir`, run this example - * `$ ./bin/run-example org.apache.spark.examples.streaming.HdfsWordCount local[2] localdir` + * `$ ./bin/spark-submit examples.jar \ + * --class org.apache.spark.examples.streaming.HdfsWordCount localdir` * Then create a text file in `localdir` and the words in the file will get counted. */ object HdfsWordCount { def main(args: Array[String]) { - if (args.length < 2) { - System.err.println("Usage: HdfsWordCount ") + if (args.length < 1) { + System.err.println("Usage: HdfsWordCount ") System.exit(1) } StreamingExamples.setStreamingLogLevels() - + val sparkConf = new SparkConf().setAppName("HdfsWordCount") // Create the context - val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + val ssc = new StreamingContext(sparkConf, Seconds(2)) // Create the FileInputDStream on the directory and use the // stream to count words in new files created - val lines = ssc.textFileStream(args(1)) + val lines = ssc.textFileStream(args(0)) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.print() diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index c3aae5af05b1c..3af806981f37a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -24,34 +24,33 @@ import kafka.producer._ import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.kafka._ +import org.apache.spark.SparkConf -// scalastyle:off /** * Consumes messages from one or more topics in Kafka and does wordcount. - * Usage: KafkaWordCount - * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * Usage: KafkaWordCount * is a list of one or more zookeeper servers that make quorum * is the name of kafka consumer group * is a list of one or more kafka topics to consume from * is the number of threads the kafka consumer should use * * Example: - * `./bin/run-example org.apache.spark.examples.streaming.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` + * `./bin/spark-submit examples.jar \ + * --class org.apache.spark.examples.streaming.KafkaWordCount local[2] zoo01,zoo02,zoo03 \ + * my-consumer-group topic1,topic2 1` */ -// scalastyle:on object KafkaWordCount { def main(args: Array[String]) { - if (args.length < 5) { - System.err.println("Usage: KafkaWordCount ") + if (args.length < 4) { + System.err.println("Usage: KafkaWordCount ") System.exit(1) } StreamingExamples.setStreamingLogLevels() - val Array(master, zkQuorum, group, topics, numThreads) = args - - val ssc = new StreamingContext(master, "KafkaWordCount", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + val Array(zkQuorum, group, topics, numThreads) = args + val sparkConf = new SparkConf().setAppName("KafkaWordCount") + val ssc = new StreamingContext(sparkConf, Seconds(2)) ssc.checkpoint("checkpoint") val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala index 47bf1e5a06439..3a10daa9ab84a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala @@ -24,6 +24,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.mqtt._ +import org.apache.spark.SparkConf /** * A simple Mqtt publisher for demonstration purposes, repeatedly publishes @@ -64,7 +65,6 @@ object MQTTPublisher { } } -// scalastyle:off /** * A sample wordcount with MqttStream stream * @@ -74,30 +74,28 @@ object MQTTPublisher { * Eclipse paho project provides Java library for Mqtt Client http://www.eclipse.org/paho/ * Example Java code for Mqtt Publisher and Subscriber can be found here * https://bitbucket.org/mkjinesh/mqttclient - * Usage: MQTTWordCount - * In local mode, should be 'local[n]' with n > 1 - * and describe where Mqtt publisher is running. + * Usage: MQTTWordCount +\ * and describe where Mqtt publisher is running. * * To run this example locally, you may run publisher as - * `$ ./bin/run-example org.apache.spark.examples.streaming.MQTTPublisher tcp://localhost:1883 foo` + * `$ ./bin/spark-submit examples.jar \ + * --class org.apache.spark.examples.streaming.MQTTPublisher tcp://localhost:1883 foo` * and run the example as - * `$ ./bin/run-example org.apache.spark.examples.streaming.MQTTWordCount local[2] tcp://localhost:1883 foo` + * `$ ./bin/spark-submit examples.jar \ + * --class org.apache.spark.examples.streaming.MQTTWordCount tcp://localhost:1883 foo` */ -// scalastyle:on object MQTTWordCount { def main(args: Array[String]) { - if (args.length < 3) { + if (args.length < 2) { System.err.println( - "Usage: MQTTWordCount " + - " In local mode, should be 'local[n]' with n > 1") + "Usage: MQTTWordCount ") System.exit(1) } - val Seq(master, brokerUrl, topic) = args.toSeq - - val ssc = new StreamingContext(master, "MqttWordCount", Seconds(2), System.getenv("SPARK_HOME"), - StreamingContext.jarOfClass(this.getClass).toSeq) + val Seq(brokerUrl, topic) = args.toSeq + val sparkConf = new SparkConf().setAppName("MQTTWordCount") + val ssc = new StreamingContext(sparkConf, Seconds(2)) val lines = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_ONLY_SER_2) val words = lines.flatMap(x => x.toString.split(" ")) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala index acfe9a4da3596..ad7a199b2c0ab 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala @@ -17,41 +17,38 @@ package org.apache.spark.examples.streaming +import org.apache.spark.SparkConf import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.storage.StorageLevel -// scalastyle:off /** * Counts words in text encoded with UTF8 received from the network every second. * - * Usage: NetworkWordCount - * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. - * and describe the TCP server that Spark Streaming would connect to receive data. + * Usage: NetworkWordCount + * and describe the TCP server that Spark Streaming would connect to receive data. * * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount local[2] localhost 9999` + * `$ ./bin/spark-submit examples.jar \ + * --class org.apache.spark.examples.streaming.NetworkWordCount localhost 9999` */ -// scalastyle:on object NetworkWordCount { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: NetworkWordCount \n" + - "In local mode, should be 'local[n]' with n > 1") + if (args.length < 2) { + System.err.println("Usage: NetworkWordCount ") System.exit(1) } StreamingExamples.setStreamingLogLevels() - + val sparkConf = new SparkConf().setAppName("NetworkWordCount"); // Create the context with a 1 second batch size - val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + val ssc = new StreamingContext(sparkConf, Seconds(1)) // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') - val lines = ssc.socketTextStream(args(1), args(2).toInt, StorageLevel.MEMORY_ONLY_SER) + val lines = ssc.socketTextStream(args(0), args(1).toInt, StorageLevel.MEMORY_ONLY_SER) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.print() diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala index f92f72f2de876..4caa90659111a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala @@ -19,6 +19,7 @@ package org.apache.spark.examples.streaming import scala.collection.mutable.SynchronizedQueue +import org.apache.spark.SparkConf import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ @@ -26,16 +27,11 @@ import org.apache.spark.streaming.StreamingContext._ object QueueStream { def main(args: Array[String]) { - if (args.length < 1) { - System.err.println("Usage: QueueStream ") - System.exit(1) - } StreamingExamples.setStreamingLogLevels() - + val sparkConf = new SparkConf().setAppName("QueueStream") // Create the context - val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + val ssc = new StreamingContext(sparkConf, Seconds(1)) // Create the queue through which RDDs can be pushed to // a QueueInputDStream diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala index 1b0319a046433..a9aaa445bccb6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala @@ -17,6 +17,7 @@ package org.apache.spark.examples.streaming +import org.apache.spark.SparkConf import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.util.IntParam @@ -27,29 +28,26 @@ import org.apache.spark.util.IntParam * will only work with spark.streaming.util.RawTextSender running on all worker nodes * and with Spark using Kryo serialization (set Java property "spark.serializer" to * "org.apache.spark.serializer.KryoSerializer"). - * Usage: RawNetworkGrep - * is the Spark master URL + * Usage: RawNetworkGrep * is the number rawNetworkStreams, which should be same as number * of work nodes in the cluster * is "localhost". * is the port on which RawTextSender is running in the worker nodes. * is the Spark Streaming batch duration in milliseconds. */ - object RawNetworkGrep { def main(args: Array[String]) { - if (args.length != 5) { - System.err.println("Usage: RawNetworkGrep ") + if (args.length != 4) { + System.err.println("Usage: RawNetworkGrep ") System.exit(1) } StreamingExamples.setStreamingLogLevels() - val Array(master, IntParam(numStreams), host, IntParam(port), IntParam(batchMillis)) = args - + val Array(IntParam(numStreams), host, IntParam(port), IntParam(batchMillis)) = args + val sparkConf = new SparkConf().setAppName("RawNetworkGrep") // Create the context - val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + val ssc = new StreamingContext(sparkConf, Duration(batchMillis)) val rawStreams = (1 to numStreams).map(_ => ssc.rawSocketStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index b0bc31cc66ab5..ace785d9fe4c5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -17,19 +17,21 @@ package org.apache.spark.examples.streaming +import java.io.File +import java.nio.charset.Charset + +import com.google.common.io.Files + +import org.apache.spark.SparkConf +import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Time, Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.util.IntParam -import java.io.File -import org.apache.spark.rdd.RDD -import com.google.common.io.Files -import java.nio.charset.Charset /** * Counts words in text encoded with UTF8 received from the network every second. * - * Usage: NetworkWordCount - * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * Usage: NetworkWordCount * and describe the TCP server that Spark Streaming would connect to receive * data. directory to HDFS-compatible file system which checkpoint data * file to which the word counts will be appended @@ -44,8 +46,9 @@ import java.nio.charset.Charset * * and run the example as * - * `$ ./run-example org.apache.spark.examples.streaming.RecoverableNetworkWordCount \ - * local[2] localhost 9999 ~/checkpoint/ ~/out` + * `$ ./bin/spark-submit examples.jar \ + * --class org.apache.spark.examples.streaming.RecoverableNetworkWordCount \ + * localhost 9999 ~/checkpoint/ ~/out` * * If the directory ~/checkpoint/ does not exist (e.g. running for the first time), it will create * a new StreamingContext (will print "Creating new context" to the console). Otherwise, if @@ -67,17 +70,16 @@ import java.nio.charset.Charset object RecoverableNetworkWordCount { - def createContext(master: String, ip: String, port: Int, outputPath: String) = { + def createContext(ip: String, port: Int, outputPath: String) = { // If you do not see this printed, that means the StreamingContext has been loaded // from the new checkpoint println("Creating new context") val outputFile = new File(outputPath) if (outputFile.exists()) outputFile.delete() - + val sparkConf = new SparkConf().setAppName("RecoverableNetworkWordCount") // Create the context with a 1 second batch size - val ssc = new StreamingContext(master, "RecoverableNetworkWordCount", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + val ssc = new StreamingContext(sparkConf, Seconds(1)) // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') @@ -94,13 +96,12 @@ object RecoverableNetworkWordCount { } def main(args: Array[String]) { - if (args.length != 5) { + if (args.length != 4) { System.err.println("You arguments were " + args.mkString("[", ", ", "]")) System.err.println( """ - |Usage: RecoverableNetworkWordCount - | is the Spark master URL. In local mode, should be - | 'local[n]' with n > 1. and describe the TCP server that Spark + |Usage: RecoverableNetworkWordCount + | . and describe the TCP server that Spark | Streaming would connect to receive data. directory to | HDFS-compatible file system which checkpoint data file to which the | word counts will be appended @@ -111,10 +112,10 @@ object RecoverableNetworkWordCount { ) System.exit(1) } - val Array(master, ip, IntParam(port), checkpointDirectory, outputPath) = args + val Array(ip, IntParam(port), checkpointDirectory, outputPath) = args val ssc = StreamingContext.getOrCreate(checkpointDirectory, () => { - createContext(master, ip, port, outputPath) + createContext(ip, port, outputPath) }) ssc.start() ssc.awaitTermination() diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala index 8001d56c98d86..5e1415f3cc536 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala @@ -17,28 +17,27 @@ package org.apache.spark.examples.streaming +import org.apache.spark.SparkConf import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ -// scalastyle:off + /** * Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every * second. - * Usage: StatefulNetworkWordCount - * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * Usage: StatefulNetworkWordCount * and describe the TCP server that Spark Streaming would connect to receive * data. * * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./bin/run-example org.apache.spark.examples.streaming.StatefulNetworkWordCount local[2] localhost 9999` + * `$ ./bin/spark-submit examples.jar + * --class org.apache.spark.examples.streaming.StatefulNetworkWordCount localhost 9999` */ -// scalastyle:on object StatefulNetworkWordCount { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: StatefulNetworkWordCount \n" + - "In local mode, should be 'local[n]' with n > 1") + if (args.length < 2) { + System.err.println("Usage: StatefulNetworkWordCount ") System.exit(1) } @@ -52,14 +51,14 @@ object StatefulNetworkWordCount { Some(currentCount + previousCount) } + val sparkConf = new SparkConf().setAppName("NetworkWordCumulativeCountUpdateStateByKey") // Create the context with a 1 second batch size - val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", - Seconds(1), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + val ssc = new StreamingContext(sparkConf, Seconds(1)) ssc.checkpoint(".") // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited test (eg. generated by 'nc') - val lines = ssc.socketTextStream(args(1), args(2).toInt) + val lines = ssc.socketTextStream(args(0), args(1).toInt) val words = lines.flatMap(_.split(" ")) val wordDstream = words.map(x => (x, 1)) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala index b12617d881787..683752ac96241 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala @@ -19,11 +19,13 @@ package org.apache.spark.examples.streaming import com.twitter.algebird._ +import org.apache.spark.SparkConf import org.apache.spark.SparkContext._ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.twitter._ + // scalastyle:off /** * Illustrates the use of the Count-Min Sketch, from Twitter's Algebird library, to compute @@ -49,12 +51,6 @@ import org.apache.spark.streaming.twitter._ // scalastyle:on object TwitterAlgebirdCMS { def main(args: Array[String]) { - if (args.length < 1) { - System.err.println("Usage: TwitterAlgebirdCMS " + - " [filter1] [filter2] ... [filter n]") - System.exit(1) - } - StreamingExamples.setStreamingLogLevels() // CMS parameters @@ -65,10 +61,9 @@ object TwitterAlgebirdCMS { // K highest frequency elements to take val TOPK = 10 - val (master, filters) = (args.head, args.tail) - - val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + val filters = args + val sparkConf = new SparkConf().setAppName("TwitterAlgebirdCMS") + val ssc = new StreamingContext(sparkConf, Seconds(10)) val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER_2) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala index 22f232c72545c..62db5e663b8af 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala @@ -23,6 +23,8 @@ import com.twitter.algebird.HyperLogLog._ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.twitter._ +import org.apache.spark.SparkConf + // scalastyle:off /** * Illustrates the use of the HyperLogLog algorithm, from Twitter's Algebird library, to compute @@ -42,20 +44,14 @@ import org.apache.spark.streaming.twitter._ // scalastyle:on object TwitterAlgebirdHLL { def main(args: Array[String]) { - if (args.length < 1) { - System.err.println("Usage: TwitterAlgebirdHLL " + - " [filter1] [filter2] ... [filter n]") - System.exit(1) - } StreamingExamples.setStreamingLogLevels() /** Bit size parameter for HyperLogLog, trades off accuracy vs size */ val BIT_SIZE = 12 - val (master, filters) = (args.head, args.tail) - - val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + val filters = args + val sparkConf = new SparkConf().setAppName("TwitterAlgebirdHLL") + val ssc = new StreamingContext(sparkConf, Seconds(5)) val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala index 5b58e94600a16..1ddff22cb8a42 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala @@ -21,6 +21,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} import StreamingContext._ import org.apache.spark.SparkContext._ import org.apache.spark.streaming.twitter._ +import org.apache.spark.SparkConf /** * Calculates popular hashtags (topics) over sliding 10 and 60 second windows from a Twitter @@ -30,18 +31,12 @@ import org.apache.spark.streaming.twitter._ */ object TwitterPopularTags { def main(args: Array[String]) { - if (args.length < 1) { - System.err.println("Usage: TwitterPopularTags " + - " [filter1] [filter2] ... [filter n]") - System.exit(1) - } StreamingExamples.setStreamingLogLevels() - val (master, filters) = (args.head, args.tail) - - val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + val filters = args + val sparkConf = new SparkConf().setAppName("TwitterPopularTags") + val ssc = new StreamingContext(sparkConf, Seconds(2)) val stream = TwitterUtils.createStream(ssc, None, filters) val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala index de46e5f5b10b6..7ade3f1018ee8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala @@ -28,6 +28,7 @@ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.zeromq._ import scala.language.implicitConversions +import org.apache.spark.SparkConf /** * A simple publisher for demonstration purposes, repeatedly publishes random Messages @@ -63,30 +64,28 @@ object SimpleZeroMQPublisher { * Install zeroMQ (release 2.1) core libraries. [ZeroMQ Install guide] * (http://www.zeromq.org/intro:get-the-software) * - * Usage: ZeroMQWordCount - * In local mode, should be 'local[n]' with n > 1 + * Usage: ZeroMQWordCount * and describe where zeroMq publisher is running. * * To run this example locally, you may run publisher as - * `$ ./bin/run-example org.apache.spark.examples.streaming.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` + * `$ ./bin/spark-submit examples.jar \ + * --class org.apache.spark.examples.streaming.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` * and run the example as - * `$ ./bin/run-example org.apache.spark.examples.streaming.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` + * `$ ./bin/spark-submit examples.jar \ + * --class org.apache.spark.examples.streaming.ZeroMQWordCount tcp://127.0.1.1:1234 foo` */ // scalastyle:on object ZeroMQWordCount { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println( - "Usage: ZeroMQWordCount " + - "In local mode, should be 'local[n]' with n > 1") + if (args.length < 2) { + System.err.println("Usage: ZeroMQWordCount ") System.exit(1) } StreamingExamples.setStreamingLogLevels() - val Seq(master, url, topic) = args.toSeq - + val Seq(url, topic) = args.toSeq + val sparkConf = new SparkConf().setAppName("ZeroMQWordCount") // Create the context and set the batch size - val ssc = new StreamingContext(master, "ZeroMQWordCount", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + val ssc = new StreamingContext(sparkConf, Seconds(2)) def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala index fa533a512d53b..d901d4fe225fe 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala @@ -27,10 +27,14 @@ import org.apache.spark.graphx.PartitionStrategy._ object Analytics extends Logging { def main(args: Array[String]): Unit = { - val host = args(0) - val taskType = args(1) - val fname = args(2) - val options = args.drop(3).map { arg => + if (args.length < 2) { + System.err.println("Usage: Analytics [other options]") + System.exit(1) + } + + val taskType = args(0) + val fname = args(1) + val options = args.drop(2).map { arg => arg.dropWhile(_ == '-').split('=') match { case Array(opt, v) => (opt -> v) case _ => throw new IllegalArgumentException("Invalid argument: " + arg) @@ -71,7 +75,7 @@ object Analytics extends Logging { println("| PageRank |") println("======================================") - val sc = new SparkContext(host, "PageRank(" + fname + ")", conf) + val sc = new SparkContext(conf.setAppName("PageRank(" + fname + ")")) val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, minEdgePartitions = numEPart).cache() @@ -115,7 +119,7 @@ object Analytics extends Logging { println("| Connected Components |") println("======================================") - val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")", conf) + val sc = new SparkContext(conf.setAppName("ConnectedComponents(" + fname + ")")) val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, minEdgePartitions = numEPart).cache() val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) @@ -137,7 +141,7 @@ object Analytics extends Logging { println("======================================") println("| Triangle Count |") println("======================================") - val sc = new SparkContext(host, "TriangleCount(" + fname + ")", conf) + val sc = new SparkContext(conf.setAppName("TriangleCount(" + fname + ")")) val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true, minEdgePartitions = numEPart).partitionBy(partitionStrategy).cache() val triangles = TriangleCount.run(graph) From c3f8b78c211df6c5adae74f37e39fb55baeff723 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 8 May 2014 12:13:07 -0700 Subject: [PATCH 282/641] [SPARK-1745] Move interrupted flag from TaskContext constructor (minor) It makes little sense to start a TaskContext that is interrupted. Indeed, I searched for all use cases of it and didn't find a single instance in which `interrupted` is true on construction. This was inspired by reviewing #640, which adds an additional `@volatile var completed` that is similar. These are not the most urgent changes, but I wanted to push them out before I forget. Author: Andrew Or Closes #675 from andrewor14/task-context and squashes the following commits: 9575e02 [Andrew Or] Add space 69455d1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into task-context c471490 [Andrew Or] Oops, removed one flag too many. Adding it back. 85311f8 [Andrew Or] Move interrupted flag from TaskContext constructor --- .../scala/org/apache/spark/TaskContext.scala | 20 ++++++++++--------- .../spark/scheduler/ShuffleMapTask.scala | 3 +-- .../java/org/apache/spark/JavaAPISuite.java | 2 +- .../org/apache/spark/CacheManagerSuite.scala | 10 +++------- .../org/apache/spark/PipedRDDSuite.scala | 4 +--- 5 files changed, 17 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index fc4812753d005..51f40c339d13c 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -28,13 +28,12 @@ import org.apache.spark.executor.TaskMetrics */ @DeveloperApi class TaskContext( - val stageId: Int, - val partitionId: Int, - val attemptId: Long, - val runningLocally: Boolean = false, - @volatile var interrupted: Boolean = false, - private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty -) extends Serializable { + val stageId: Int, + val partitionId: Int, + val attemptId: Long, + val runningLocally: Boolean = false, + private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty) + extends Serializable { @deprecated("use partitionId", "0.8.1") def splitId = partitionId @@ -42,7 +41,10 @@ class TaskContext( // List of callback functions to execute when the task completes. @transient private val onCompleteCallbacks = new ArrayBuffer[() => Unit] - // Set to true when the task is completed, before the onCompleteCallbacks are executed. + // Whether the corresponding task has been killed. + @volatile var interrupted: Boolean = false + + // Whether the task has completed, before the onCompleteCallbacks are executed. @volatile var completed: Boolean = false /** @@ -58,6 +60,6 @@ class TaskContext( def executeOnCompleteCallbacks() { completed = true // Process complete callbacks in the reverse order of registration - onCompleteCallbacks.reverse.foreach{_()} + onCompleteCallbacks.reverse.foreach { _() } } } 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 2259df0b56bad..4b0324f2b5447 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -23,7 +23,6 @@ import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashMap -import scala.util.Try import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics @@ -70,7 +69,7 @@ private[spark] object ShuffleMapTask { } // Since both the JarSet and FileSet have the same format this is used for both. - def deserializeFileSet(bytes: Array[Byte]) : HashMap[String, Long] = { + def deserializeFileSet(bytes: Array[Byte]): HashMap[String, Long] = { val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) val objIn = new ObjectInputStream(in) val set = objIn.readObject().asInstanceOf[Array[(String, Long)]].toMap diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index c3e03cea917b3..1912015827927 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -597,7 +597,7 @@ public void persist() { @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); - TaskContext context = new TaskContext(0, 0, 0, false, false, new TaskMetrics()); + TaskContext context = new TaskContext(0, 0, 0, false, new TaskMetrics()); Assert.assertEquals(1, rdd.iterator(rdd.splits().get(0), context).next().intValue()); } diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index fd5b0906e6765..4f178db40f638 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -23,7 +23,6 @@ import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.mock.EasyMockSugar import org.apache.spark.rdd.RDD -import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage._ // TODO: Test the CacheManager's thread-safety aspects @@ -59,8 +58,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } whenExecuting(blockManager) { - val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = TaskMetrics.empty) + val context = new TaskContext(0, 0, 0) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } @@ -72,8 +70,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } whenExecuting(blockManager) { - val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = TaskMetrics.empty) + val context = new TaskContext(0, 0, 0) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(5, 6, 7)) } @@ -86,8 +83,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } whenExecuting(blockManager) { - val context = new TaskContext(0, 0, 0, runningLocally = true, interrupted = false, - taskMetrics = TaskMetrics.empty) + val context = new TaskContext(0, 0, 0, runningLocally = true) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index 0bb6a6b09c5b5..db56a4acdd6f5 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -178,14 +178,12 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { } val hadoopPart1 = generateFakeHadoopPartition() val pipedRdd = new PipedRDD(nums, "printenv " + varName) - val tContext = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = TaskMetrics.empty) + val tContext = new TaskContext(0, 0, 0) val rddIter = pipedRdd.compute(hadoopPart1, tContext) val arr = rddIter.toArray assert(arr(0) == "/some/path") } else { // printenv isn't available so just pass the test - assert(true) } } From 5c5e7d5809d337ce41a7a90eb9201e12803aba48 Mon Sep 17 00:00:00 2001 From: Evan Sparks Date: Thu, 8 May 2014 13:07:30 -0700 Subject: [PATCH 283/641] Fixing typo in als.py XtY should be Xty. Author: Evan Sparks Closes #696 from etrain/patch-2 and squashes the following commits: 634cb8d [Evan Sparks] Fixing typo in als.py --- examples/src/main/python/als.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 33700ab4f8c53..01552dc1d449e 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -38,7 +38,7 @@ def update(i, vec, mat, ratings): ff = mat.shape[1] XtX = mat.T * mat - XtY = mat.T * ratings[i, :].T + Xty = mat.T * ratings[i, :].T for j in range(ff): XtX[j,j] += LAMBDA * uu From 322b1808d21143dc323493203929488d69e8878a Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 8 May 2014 15:31:47 -0700 Subject: [PATCH 284/641] [SPARK-1754] [SQL] Add missing arithmetic DSL operations. Add missing arithmetic DSL operations: `unary_-`, `%`. Author: Takuya UESHIN Closes #689 from ueshin/issues/SPARK-1754 and squashes the following commits: a09ef69 [Takuya UESHIN] Add also missing ! (not) operation. f73ae2c [Takuya UESHIN] Remove redundant tests. 5b3f087 [Takuya UESHIN] Add tests relating DSL operations. e09c5b8 [Takuya UESHIN] Add missing arithmetic DSL operations. --- .../apache/spark/sql/catalyst/dsl/package.scala | 4 ++++ .../expressions/ExpressionEvaluationSuite.scala | 16 +++++++++++++++- 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index dc83485df195c..78d3a1d8096af 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -57,10 +57,14 @@ package object dsl { trait ImplicitOperators { def expr: Expression + def unary_- = UnaryMinus(expr) + def unary_! = Not(expr) + def + (other: Expression) = Add(expr, other) def - (other: Expression) = Subtract(expr, other) def * (other: Expression) = Multiply(expr, other) def / (other: Expression) = Divide(expr, other) + def % (other: Expression) = Remainder(expr, other) def && (other: Expression) = And(expr, other) def || (other: Expression) = Or(expr, other) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 91605d0a260e5..344d8a304fc11 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -61,7 +61,7 @@ class ExpressionEvaluationSuite extends FunSuite { test("3VL Not") { notTrueTable.foreach { case (v, answer) => - val expr = Not(Literal(v, BooleanType)) + val expr = ! Literal(v, BooleanType) val result = expr.eval(null) if (result != answer) fail(s"$expr should not evaluate to $result, expected: $answer") } @@ -381,6 +381,13 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Add(c1, Literal(null, IntegerType)), null, row) checkEvaluation(Add(Literal(null, IntegerType), c2), null, row) checkEvaluation(Add(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + + checkEvaluation(-c1, -1, row) + checkEvaluation(c1 + c2, 3, row) + checkEvaluation(c1 - c2, -1, row) + checkEvaluation(c1 * c2, 2, row) + checkEvaluation(c1 / c2, 0, row) + checkEvaluation(c1 % c2, 1, row) } test("BinaryComparison") { @@ -395,6 +402,13 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(LessThan(c1, Literal(null, IntegerType)), null, row) checkEvaluation(LessThan(Literal(null, IntegerType), c2), null, row) checkEvaluation(LessThan(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + + checkEvaluation(c1 < c2, true, row) + checkEvaluation(c1 <= c2, true, row) + checkEvaluation(c1 > c2, false, row) + checkEvaluation(c1 >= c2, false, row) + checkEvaluation(c1 === c2, false, row) + checkEvaluation(c1 !== c2, true, row) } } From d38febee46ed156b0c8ec64757db6c290e488421 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Thu, 8 May 2014 17:52:32 -0700 Subject: [PATCH 285/641] MLlib documentation fix Fixed the documentation for that `loadLibSVMData` is changed to `loadLibSVMFile`. Author: DB Tsai Closes #703 from dbtsai/dbtsai-docfix and squashes the following commits: 71dd508 [DB Tsai] loadLibSVMData is changed to loadLibSVMFile --- docs/mllib-basics.md | 8 ++++---- docs/mllib-linear-methods.md | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/mllib-basics.md b/docs/mllib-basics.md index 704308802d65b..aa9321a547097 100644 --- a/docs/mllib-basics.md +++ b/docs/mllib-basics.md @@ -184,7 +184,7 @@ After loading, the feature indices are converted to zero-based.
      -[`MLUtils.loadLibSVMData`](api/mllib/index.html#org.apache.spark.mllib.util.MLUtils$) reads training +[`MLUtils.loadLibSVMFile`](api/mllib/index.html#org.apache.spark.mllib.util.MLUtils$) reads training examples stored in LIBSVM format. {% highlight scala %} @@ -192,12 +192,12 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD -val training: RDD[LabeledPoint] = MLUtils.loadLibSVMData(sc, "mllib/data/sample_libsvm_data.txt") +val training: RDD[LabeledPoint] = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") {% endhighlight %}
      -[`MLUtils.loadLibSVMData`](api/mllib/index.html#org.apache.spark.mllib.util.MLUtils$) reads training +[`MLUtils.loadLibSVMFile`](api/mllib/index.html#org.apache.spark.mllib.util.MLUtils$) reads training examples stored in LIBSVM format. {% highlight java %} @@ -205,7 +205,7 @@ import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.mllib.util.MLUtils; import org.apache.spark.rdd.RDDimport; -RDD training = MLUtils.loadLibSVMData(jsc, "mllib/data/sample_libsvm_data.txt"); +RDD training = MLUtils.loadLibSVMFile(jsc, "mllib/data/sample_libsvm_data.txt"); {% endhighlight %}
      diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 40b7a7f80708c..eff617d8641e2 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -186,7 +186,7 @@ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLUtils // Load training data in LIBSVM format. -val data = MLUtils.loadLibSVMData(sc, "mllib/data/sample_libsvm_data.txt") +val data = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") // Split data into training (60%) and test (40%). val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) From 910a13b3c52a6309068b4997da6df6b7d6058a1b Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Thu, 8 May 2014 17:53:22 -0700 Subject: [PATCH 286/641] [SPARK-1157][MLlib] Bug fix: lossHistory should exclude rejection steps, and remove miniBatch Getting the lossHistory from Breeze's API which already excludes the rejection steps in line search. Also, remove the miniBatch in LBFGS since those quasi-Newton methods approximate the inverse of Hessian. It doesn't make sense if the gradients are computed from a varying objective. Author: DB Tsai Closes #582 from dbtsai/dbtsai-lbfgs-bug and squashes the following commits: 9cc6cf9 [DB Tsai] Removed the miniBatch in LBFGS. 1ba6a33 [DB Tsai] Formatting the code. d72c679 [DB Tsai] Using Breeze's states to get the loss. --- .../spark/mllib/optimization/LBFGS.scala | 63 ++++++++----------- .../spark/mllib/optimization/LBFGSSuite.scala | 15 ++--- 2 files changed, 30 insertions(+), 48 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index 969a0c5f7c953..8f187c9df5102 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -42,7 +42,6 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) private var convergenceTol = 1E-4 private var maxNumIterations = 100 private var regParam = 0.0 - private var miniBatchFraction = 1.0 /** * Set the number of corrections used in the LBFGS update. Default 10. @@ -57,14 +56,6 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) this } - /** - * Set fraction of data to be used for each L-BFGS iteration. Default 1.0. - */ - def setMiniBatchFraction(fraction: Double): this.type = { - this.miniBatchFraction = fraction - this - } - /** * Set the convergence tolerance of iterations for L-BFGS. Default 1E-4. * Smaller value will lead to higher accuracy with the cost of more iterations. @@ -110,7 +101,7 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) } override def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector = { - val (weights, _) = LBFGS.runMiniBatchLBFGS( + val (weights, _) = LBFGS.runLBFGS( data, gradient, updater, @@ -118,7 +109,6 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) convergenceTol, maxNumIterations, regParam, - miniBatchFraction, initialWeights) weights } @@ -132,10 +122,8 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) @DeveloperApi object LBFGS extends Logging { /** - * Run Limited-memory BFGS (L-BFGS) in parallel using mini batches. - * In each iteration, we sample a subset (fraction miniBatchFraction) of the total data - * in order to compute a gradient estimate. - * Sampling, and averaging the subgradients over this subset is performed using one standard + * Run Limited-memory BFGS (L-BFGS) in parallel. + * Averaging the subgradients over different partitions is performed using one standard * spark map-reduce in each iteration. * * @param data - Input data for L-BFGS. RDD of the set of data examples, each of @@ -147,14 +135,12 @@ object LBFGS extends Logging { * @param convergenceTol - The convergence tolerance of iterations for L-BFGS * @param maxNumIterations - Maximal number of iterations that L-BFGS can be run. * @param regParam - Regularization parameter - * @param miniBatchFraction - Fraction of the input data set that should be used for - * one iteration of L-BFGS. Default value 1.0. * * @return A tuple containing two elements. The first element is a column matrix containing * weights for every feature, and the second element is an array containing the loss * computed for every iteration. */ - def runMiniBatchLBFGS( + def runLBFGS( data: RDD[(Double, Vector)], gradient: Gradient, updater: Updater, @@ -162,23 +148,33 @@ object LBFGS extends Logging { convergenceTol: Double, maxNumIterations: Int, regParam: Double, - miniBatchFraction: Double, initialWeights: Vector): (Vector, Array[Double]) = { val lossHistory = new ArrayBuffer[Double](maxNumIterations) val numExamples = data.count() - val miniBatchSize = numExamples * miniBatchFraction val costFun = - new CostFun(data, gradient, updater, regParam, miniBatchFraction, lossHistory, miniBatchSize) + new CostFun(data, gradient, updater, regParam, numExamples) val lbfgs = new BreezeLBFGS[BDV[Double]](maxNumIterations, numCorrections, convergenceTol) - val weights = Vectors.fromBreeze( - lbfgs.minimize(new CachedDiffFunction(costFun), initialWeights.toBreeze.toDenseVector)) + val states = + lbfgs.iterations(new CachedDiffFunction(costFun), initialWeights.toBreeze.toDenseVector) + + /** + * NOTE: lossSum and loss is computed using the weights from the previous iteration + * and regVal is the regularization value computed in the previous iteration as well. + */ + var state = states.next() + while(states.hasNext) { + lossHistory.append(state.value) + state = states.next() + } + lossHistory.append(state.value) + val weights = Vectors.fromBreeze(state.x) - logInfo("LBFGS.runMiniBatchSGD finished. Last 10 losses %s".format( + logInfo("LBFGS.runLBFGS finished. Last 10 losses %s".format( lossHistory.takeRight(10).mkString(", "))) (weights, lossHistory.toArray) @@ -193,9 +189,7 @@ object LBFGS extends Logging { gradient: Gradient, updater: Updater, regParam: Double, - miniBatchFraction: Double, - lossHistory: ArrayBuffer[Double], - miniBatchSize: Double) extends DiffFunction[BDV[Double]] { + numExamples: Long) extends DiffFunction[BDV[Double]] { private var i = 0 @@ -204,8 +198,7 @@ object LBFGS extends Logging { val localData = data val localGradient = gradient - val (gradientSum, lossSum) = localData.sample(false, miniBatchFraction, 42 + i) - .aggregate((BDV.zeros[Double](weights.size), 0.0))( + val (gradientSum, lossSum) = localData.aggregate((BDV.zeros[Double](weights.size), 0.0))( seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => val l = localGradient.compute( features, label, Vectors.fromBreeze(weights), Vectors.fromBreeze(grad)) @@ -223,7 +216,7 @@ object LBFGS extends Logging { Vectors.fromBreeze(weights), Vectors.dense(new Array[Double](weights.size)), 0, 1, regParam)._2 - val loss = lossSum / miniBatchSize + regVal + val loss = lossSum / numExamples + regVal /** * It will return the gradient part of regularization using updater. * @@ -245,14 +238,8 @@ object LBFGS extends Logging { Vectors.fromBreeze(weights), Vectors.dense(new Array[Double](weights.size)), 1, 1, regParam)._1.toBreeze - // gradientTotal = gradientSum / miniBatchSize + gradientTotal - axpy(1.0 / miniBatchSize, gradientSum, gradientTotal) - - /** - * NOTE: lossSum and loss is computed using the weights from the previous iteration - * and regVal is the regularization value computed in the previous iteration as well. - */ - lossHistory.append(loss) + // gradientTotal = gradientSum / numExamples + gradientTotal + axpy(1.0 / numExamples, gradientSum, gradientTotal) i += 1 diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala index f33770aed30bd..6af1b502eb4dd 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala @@ -59,7 +59,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with ShouldMatchers { val convergenceTol = 1e-12 val maxNumIterations = 10 - val (_, loss) = LBFGS.runMiniBatchLBFGS( + val (_, loss) = LBFGS.runLBFGS( dataRDD, gradient, simpleUpdater, @@ -67,7 +67,6 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with ShouldMatchers { convergenceTol, maxNumIterations, regParam, - miniBatchFrac, initialWeightsWithIntercept) // Since the cost function is convex, the loss is guaranteed to be monotonically decreasing @@ -104,7 +103,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with ShouldMatchers { val convergenceTol = 1e-12 val maxNumIterations = 10 - val (weightLBFGS, lossLBFGS) = LBFGS.runMiniBatchLBFGS( + val (weightLBFGS, lossLBFGS) = LBFGS.runLBFGS( dataRDD, gradient, squaredL2Updater, @@ -112,7 +111,6 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with ShouldMatchers { convergenceTol, maxNumIterations, regParam, - miniBatchFrac, initialWeightsWithIntercept) val numGDIterations = 50 @@ -150,7 +148,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with ShouldMatchers { val maxNumIterations = 8 var convergenceTol = 0.0 - val (_, lossLBFGS1) = LBFGS.runMiniBatchLBFGS( + val (_, lossLBFGS1) = LBFGS.runLBFGS( dataRDD, gradient, squaredL2Updater, @@ -158,7 +156,6 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with ShouldMatchers { convergenceTol, maxNumIterations, regParam, - miniBatchFrac, initialWeightsWithIntercept) // Note that the first loss is computed with initial weights, @@ -166,7 +163,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with ShouldMatchers { assert(lossLBFGS1.length == 9) convergenceTol = 0.1 - val (_, lossLBFGS2) = LBFGS.runMiniBatchLBFGS( + val (_, lossLBFGS2) = LBFGS.runLBFGS( dataRDD, gradient, squaredL2Updater, @@ -174,7 +171,6 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with ShouldMatchers { convergenceTol, maxNumIterations, regParam, - miniBatchFrac, initialWeightsWithIntercept) // Based on observation, lossLBFGS2 runs 3 iterations, no theoretically guaranteed. @@ -182,7 +178,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with ShouldMatchers { assert((lossLBFGS2(2) - lossLBFGS2(3)) / lossLBFGS2(2) < convergenceTol) convergenceTol = 0.01 - val (_, lossLBFGS3) = LBFGS.runMiniBatchLBFGS( + val (_, lossLBFGS3) = LBFGS.runLBFGS( dataRDD, gradient, squaredL2Updater, @@ -190,7 +186,6 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with ShouldMatchers { convergenceTol, maxNumIterations, regParam, - miniBatchFrac, initialWeightsWithIntercept) // With smaller convergenceTol, it takes more steps. From 191279ce4edb940821d11a6b25cd33c8ad0af054 Mon Sep 17 00:00:00 2001 From: Funes Date: Thu, 8 May 2014 17:54:10 -0700 Subject: [PATCH 287/641] Bug fix of sparse vector conversion Fixed a small bug caused by the inconsistency of index/data array size and vector length. Author: Funes Author: funes Closes #661 from funes/bugfix and squashes the following commits: edb2b9d [funes] remove unused import 75dced3 [Funes] update test case d129a66 [Funes] Add test for sparse breeze by vector builder 64e7198 [Funes] Copy data only when necessary b85806c [Funes] Bug fix of sparse vector conversion --- .../scala/org/apache/spark/mllib/linalg/Vectors.scala | 6 +++++- .../spark/mllib/linalg/BreezeVectorConversionSuite.scala | 9 +++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 7cdf6bd56acd9..84d223908c1f6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -136,7 +136,11 @@ object Vectors { new DenseVector(v.toArray) // Can't use underlying array directly, so make a new one } case v: BSV[Double] => - new SparseVector(v.length, v.index, v.data) + if (v.index.length == v.used) { + new SparseVector(v.length, v.index, v.data) + } else { + new SparseVector(v.length, v.index.slice(0, v.used), v.data.slice(0, v.used)) + } case v: BV[_] => sys.error("Unsupported Breeze vector type: " + v.getClass.getName) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala index aacaa300849aa..8abdac72902c6 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala @@ -55,4 +55,13 @@ class BreezeVectorConversionSuite extends FunSuite { assert(vec.indices.eq(indices), "should not copy data") assert(vec.values.eq(values), "should not copy data") } + + test("sparse breeze with partially-used arrays to vector") { + val activeSize = 3 + val breeze = new BSV[Double](indices, values, activeSize, n) + val vec = Vectors.fromBreeze(breeze).asInstanceOf[SparseVector] + assert(vec.size === n) + assert(vec.indices === indices.slice(0, activeSize)) + assert(vec.values === values.slice(0, activeSize)) + } } From 2fd2752e572921a9010614eb1c1238c493d34a7c Mon Sep 17 00:00:00 2001 From: Bouke van der Bijl Date: Thu, 8 May 2014 20:43:37 -0700 Subject: [PATCH 288/641] Include the sbin/spark-config.sh in spark-executor This is needed because broadcast values are broken on pyspark on Mesos, it tries to import pyspark but can't, as the PYTHONPATH is not set due to changes in ff5be9a4 https://issues.apache.org/jira/browse/SPARK-1725 Author: Bouke van der Bijl Closes #651 from bouk/include-spark-config-in-mesos-executor and squashes the following commits: b2f1295 [Bouke van der Bijl] Inline PYTHONPATH in spark-executor eedbbcc [Bouke van der Bijl] Include the sbin/spark-config.sh in spark-executor --- sbin/spark-executor | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sbin/spark-executor b/sbin/spark-executor index de5bfab563125..336549f29c9ce 100755 --- a/sbin/spark-executor +++ b/sbin/spark-executor @@ -19,5 +19,8 @@ FWDIR="$(cd `dirname $0`/..; pwd)" +export PYTHONPATH=$FWDIR/python:$PYTHONPATH +export PYTHONPATH=$FWDIR/python/lib/py4j-0.8.1-src.zip:$PYTHONPATH + echo "Running spark-executor with framework dir = $FWDIR" exec $FWDIR/bin/spark-class org.apache.spark.executor.MesosExecutorBackend From 8b7841299439b7dc590b2f7e2339f24e8f3e19f6 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 8 May 2014 20:45:29 -0700 Subject: [PATCH 289/641] [SPARK-1755] Respect SparkSubmit --name on YARN Right now, SparkSubmit ignores the `--name` flag for both yarn-client and yarn-cluster. This is a bug. In client mode, SparkSubmit treats `--name` as a [cluster config](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala#L170) and does not propagate this to SparkContext. In cluster mode, SparkSubmit passes this flag to `org.apache.spark.deploy.yarn.Client`, which only uses it for the [YARN ResourceManager](https://github.com/apache/spark/blob/master/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala#L80), but does not propagate this to SparkContext. This PR ensures that `spark.app.name` is always set if SparkSubmit receives the `--name` flag, which is what the usage promises. This makes it possible for applications to start a SparkContext with an empty conf `val sc = new SparkContext(new SparkConf)`, and inherit the app name from SparkSubmit. Tested both modes on a YARN cluster. Author: Andrew Or Closes #699 from andrewor14/yarn-app-name and squashes the following commits: 98f6a79 [Andrew Or] Fix tests dea932f [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-app-name c86d9ca [Andrew Or] Respect SparkSubmit --name on YARN --- .../scala/org/apache/spark/deploy/SparkSubmit.scala | 9 +++++---- .../org/apache/spark/deploy/SparkSubmitSuite.scala | 10 ++++++---- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index e39723f38347c..16de6f7cdb100 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -160,6 +160,7 @@ object SparkSubmit { // each deploy mode; we iterate through these below val options = List[OptionAssigner]( OptionAssigner(args.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), + OptionAssigner(args.name, ALL_CLUSTER_MGRS, false, sysProp = "spark.app.name"), OptionAssigner(args.driverExtraClassPath, STANDALONE | YARN, true, sysProp = "spark.driver.extraClassPath"), OptionAssigner(args.driverExtraJavaOptions, STANDALONE | YARN, true, @@ -167,7 +168,7 @@ object SparkSubmit { OptionAssigner(args.driverExtraLibraryPath, STANDALONE | YARN, true, sysProp = "spark.driver.extraLibraryPath"), OptionAssigner(args.driverMemory, YARN, true, clOption = "--driver-memory"), - OptionAssigner(args.name, YARN, true, clOption = "--name"), + OptionAssigner(args.name, YARN, true, clOption = "--name", sysProp = "spark.app.name"), OptionAssigner(args.queue, YARN, true, clOption = "--queue"), OptionAssigner(args.queue, YARN, false, sysProp = "spark.yarn.queue"), OptionAssigner(args.numExecutors, YARN, true, clOption = "--num-executors"), @@ -188,8 +189,7 @@ object SparkSubmit { OptionAssigner(args.jars, YARN, true, clOption = "--addJars"), OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.files"), OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), - OptionAssigner(args.jars, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.jars"), - OptionAssigner(args.name, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.app.name") + OptionAssigner(args.jars, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.jars") ) // For client mode make any added jars immediately visible on the classpath @@ -205,7 +205,8 @@ object SparkSubmit { (clusterManager & opt.clusterManager) != 0) { if (opt.clOption != null) { childArgs += (opt.clOption, opt.value) - } else if (opt.sysProp != null) { + } + if (opt.sysProp != null) { sysProps.put(opt.sysProp, opt.value) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index d7e3b22ed476e..c9edb03cdeb0f 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -104,7 +104,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", "--driver-memory", "4g", "--queue", "thequeue", "--files", "file1.txt,file2.txt", - "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", + "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", "--name", "beauty", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) @@ -122,7 +122,8 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { childArgsStr should include ("--num-executors 6") mainClass should be ("org.apache.spark.deploy.yarn.Client") classpath should have length (0) - sysProps should have size (1) + sysProps("spark.app.name") should be ("beauty") + sysProps("SPARK_SUBMIT") should be ("true") } test("handles YARN client mode") { @@ -130,8 +131,8 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", "--driver-memory", "4g", "--queue", "thequeue", "--files", "file1.txt,file2.txt", - "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", "thejar.jar", - "arg1", "arg2") + "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", "--name", "trill", + "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -140,6 +141,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { classpath should contain ("one.jar") classpath should contain ("two.jar") classpath should contain ("three.jar") + sysProps("spark.app.name") should be ("trill") sysProps("spark.executor.memory") should be ("5g") sysProps("spark.executor.cores") should be ("5") sysProps("spark.yarn.queue") should be ("thequeue") From 3f779d872d8459b262b3db9e4d12b011910b6ce9 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 8 May 2014 20:46:11 -0700 Subject: [PATCH 290/641] [SPARK-1631] Correctly set the Yarn app name when launching the AM. Author: Marcelo Vanzin Closes #539 from vanzin/yarn-app-name and squashes the following commits: 7d1ca4f [Marcelo Vanzin] [SPARK-1631] Correctly set the Yarn app name when launching the AM. --- .../scheduler/cluster/YarnClientSchedulerBackend.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 ce2dde0631ed9..2924189077b7d 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 @@ -35,10 +35,10 @@ private[spark] class YarnClientSchedulerBackend( private[spark] def addArg(optionName: String, envVar: String, sysProp: String, arrayBuf: ArrayBuffer[String]) { - if (System.getProperty(sysProp) != null) { - arrayBuf += (optionName, System.getProperty(sysProp)) - } else if (System.getenv(envVar) != null) { + if (System.getenv(envVar) != null) { arrayBuf += (optionName, System.getenv(envVar)) + } else if (sc.getConf.contains(sysProp)) { + arrayBuf += (optionName, sc.getConf.get(sysProp)) } } From 06b15baab25951d124bbe6b64906f4139e037deb Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 8 May 2014 22:26:17 -0700 Subject: [PATCH 291/641] SPARK-1565 (Addendum): Replace `run-example` with `spark-submit`. Gives a nicely formatted message to the user when `run-example` is run to tell them to use `spark-submit`. Author: Patrick Wendell Closes #704 from pwendell/examples and squashes the following commits: 1996ee8 [Patrick Wendell] Feedback form Andrew 3eb7803 [Patrick Wendell] Suggestions from TD 2474668 [Patrick Wendell] SPARK-1565 (Addendum): Replace `run-example` with `spark-submit`. --- README.md | 19 +++-- bin/pyspark | 2 +- bin/run-example | 71 +++++-------------- bin/spark-class | 2 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 4 +- docs/running-on-yarn.md | 2 +- make-distribution.sh | 2 + 7 files changed, 37 insertions(+), 65 deletions(-) diff --git a/README.md b/README.md index e2d1dcb5672ff..9c2e32b90f162 100644 --- a/README.md +++ b/README.md @@ -39,17 +39,22 @@ And run the following command, which should also return 1000: ## Example Programs Spark also comes with several sample programs in the `examples` directory. -To run one of them, use `./bin/run-example `. For example: +To run one of them, use `./bin/run-example [params]`. For example: - ./bin/run-example org.apache.spark.examples.SparkLR local[2] + ./bin/run-example org.apache.spark.examples.SparkLR -will run the Logistic Regression example locally on 2 CPUs. +will run the Logistic Regression example locally. -Each of the example programs prints usage help if no params are given. +You can set the MASTER environment variable when running examples to submit +examples to a cluster. This can be a mesos:// or spark:// URL, +"yarn-cluster" or "yarn-client" to run on YARN, and "local" to run +locally with one thread, or "local[N]" to run locally with N threads. You +can also use an abbreviated class name if the class is in the `examples` +package. For instance: -All of the Spark samples take a `` parameter that is the cluster URL -to connect to. This can be a mesos:// or spark:// URL, or "local" to run -locally with one thread, or "local[N]" to run locally with N threads. + MASTER=spark://host:7077 ./bin/run-example SparkPi + +Many of the example programs print usage help if no params are given. ## Running Tests diff --git a/bin/pyspark b/bin/pyspark index f5558853e8a4e..10e35e0f1734e 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -31,7 +31,7 @@ if [ ! -f "$FWDIR/RELEASE" ]; then ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null if [[ $? != 0 ]]; then echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2 - echo "You need to build Spark with sbt/sbt assembly before running this program" >&2 + echo "You need to build Spark before running this program" >&2 exit 1 fi fi diff --git a/bin/run-example b/bin/run-example index d8a94f2e31e07..146951ac0ee56 100755 --- a/bin/run-example +++ b/bin/run-example @@ -17,28 +17,10 @@ # limitations under the License. # -cygwin=false -case "`uname`" in - CYGWIN*) cygwin=true;; -esac - SCALA_VERSION=2.10 -# Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`/..; pwd)" - -# Export this as SPARK_HOME export SPARK_HOME="$FWDIR" - -. $FWDIR/bin/load-spark-env.sh - -if [ -z "$1" ]; then - echo "Usage: run-example []" >&2 - exit 1 -fi - -# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack -# to avoid the -sources and -doc packages that are built by publish-local. EXAMPLES_DIR="$FWDIR"/examples if [ -f "$FWDIR/RELEASE" ]; then @@ -49,46 +31,29 @@ fi if [[ -z $SPARK_EXAMPLES_JAR ]]; then echo "Failed to find Spark examples assembly in $FWDIR/lib or $FWDIR/examples/target" >&2 - echo "You need to build Spark with sbt/sbt assembly before running this program" >&2 + echo "You need to build Spark before running this program" >&2 exit 1 fi +EXAMPLE_MASTER=${MASTER:-"local[*]"} -# Since the examples JAR ideally shouldn't include spark-core (that dependency should be -# "provided"), also add our standard Spark classpath, built using compute-classpath.sh. -CLASSPATH=`$FWDIR/bin/compute-classpath.sh` -CLASSPATH="$SPARK_EXAMPLES_JAR:$CLASSPATH" - -if $cygwin; then - CLASSPATH=`cygpath -wp $CLASSPATH` - export SPARK_EXAMPLES_JAR=`cygpath -w $SPARK_EXAMPLES_JAR` -fi - -# Find java binary -if [ -n "${JAVA_HOME}" ]; then - RUNNER="${JAVA_HOME}/bin/java" -else - if [ `command -v java` ]; then - RUNNER="java" - else - echo "JAVA_HOME is not set" >&2 - exit 1 - fi -fi - -# Set JAVA_OPTS to be able to load native libraries and to set heap size -JAVA_OPTS="$SPARK_JAVA_OPTS" -# Load extra JAVA_OPTS from conf/java-opts, if it exists -if [ -e "$FWDIR/conf/java-opts" ] ; then - JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" +if [ -n "$1" ]; then + EXAMPLE_CLASS="$1" + shift +else + echo "usage: ./bin/run-example [example-args]" + echo " - set MASTER=XX to use a specific master" + echo " - can use abbreviated example class name (e.g. SparkPi, mllib.MovieLensALS)" + echo + exit -1 fi -export JAVA_OPTS -if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then - echo -n "Spark Command: " - echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" - echo "========================================" - echo +if [[ ! $EXAMPLE_CLASS == org.apache.spark.examples* ]]; then + EXAMPLE_CLASS="org.apache.spark.examples.$EXAMPLE_CLASS" fi -exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" +./bin/spark-submit \ + --master $EXAMPLE_MASTER \ + --class $EXAMPLE_CLASS \ + $SPARK_EXAMPLES_JAR \ + "$@" diff --git a/bin/spark-class b/bin/spark-class index 72f8b9bf9a495..6480ccb58d6aa 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -114,7 +114,7 @@ if [ ! -f "$FWDIR/RELEASE" ]; then jars_list=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar") if [ "$num_jars" -eq "0" ]; then echo "Failed to find Spark assembly in $FWDIR/assembly/target/scala-$SCALA_VERSION/" >&2 - echo "You need to build Spark with 'sbt/sbt assembly' before running this program." >&2 + echo "You need to build Spark before running this program." >&2 exit 1 fi if [ "$num_jars" -gt "1" ]; then diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index a1ca612cc9a09..9d8d8044f07eb 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -330,9 +330,9 @@ abstract class RDD[T: ClassTag]( if (shuffle) { // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( - new ShuffledRDD[T, Null, (T, Null)](map(x => (x, null)), + new ShuffledRDD[Int, T, (Int, T)](map(x => (Utils.random.nextInt(), x)), new HashPartitioner(numPartitions)), - numPartitions).keys + numPartitions).values } else { new CoalescedRDD(this, numPartitions) } diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 68183ee8b4613..c563594296802 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -53,7 +53,7 @@ For example: --driver-memory 4g \ --executor-memory 2g \ --executor-cores 1 - examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ + lib/spark-examples*.jar \ yarn-cluster 5 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. diff --git a/make-distribution.sh b/make-distribution.sh index 759e555b4b69a..1cc2844703fbb 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -40,6 +40,8 @@ # set -o pipefail +set -e + # Figure out where the Spark framework is installed FWDIR="$(cd `dirname $0`; pwd)" DISTDIR="$FWDIR/dist" From 7db47c463fefc244e9c100d4aab90451c3828261 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Thu, 8 May 2014 22:30:17 -0700 Subject: [PATCH 292/641] SPARK-1775: Unneeded lock in ShuffleMapTask.deserializeInfo This was used in the past to have a cache of deserialized ShuffleMapTasks, but that's been removed, so there's no need for a lock. It slows down Spark when task descriptions are large, e.g. due to large lineage graphs or local variables. Author: Sandeep Closes #707 from techaddict/SPARK-1775 and squashes the following commits: 18d8ebf [Sandeep] SPARK-1775: Unneeded lock in ShuffleMapTask.deserializeInfo This was used in the past to have a cache of deserialized ShuffleMapTasks, but that's been removed, so there's no need for a lock. It slows down Spark when task descriptions are large, e.g. due to large lineage graphs or local variables. --- .../apache/spark/scheduler/ShuffleMapTask.scala | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) 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 4b0324f2b5447..9ba586f7581cf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -57,15 +57,13 @@ private[spark] object ShuffleMapTask { } def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], ShuffleDependency[_,_]) = { - synchronized { - val loader = Thread.currentThread.getContextClassLoader - val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) - val ser = SparkEnv.get.closureSerializer.newInstance() - val objIn = ser.deserializeStream(in) - val rdd = objIn.readObject().asInstanceOf[RDD[_]] - val dep = objIn.readObject().asInstanceOf[ShuffleDependency[_,_]] - (rdd, dep) - } + val loader = Thread.currentThread.getContextClassLoader + val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) + val ser = SparkEnv.get.closureSerializer.newInstance() + val objIn = ser.deserializeStream(in) + val rdd = objIn.readObject().asInstanceOf[RDD[_]] + val dep = objIn.readObject().asInstanceOf[ShuffleDependency[_,_]] + (rdd, dep) } // Since both the JarSet and FileSet have the same format this is used for both. From 4c60fd1e8c526278b7e5544d6164050d1aee0338 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 8 May 2014 22:33:06 -0700 Subject: [PATCH 293/641] MINOR: Removing dead code. Meant to do this when patching up the last merge. --- .../main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala | 1 - 1 file changed, 1 deletion(-) 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 9ba586f7581cf..ed0f56f1abdf5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -57,7 +57,6 @@ private[spark] object ShuffleMapTask { } def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], ShuffleDependency[_,_]) = { - val loader = Thread.currentThread.getContextClassLoader val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) val ser = SparkEnv.get.closureSerializer.newInstance() val objIn = ser.deserializeStream(in) From 32868f31f88aebd580ab9329dc51a30c26af7a74 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 8 May 2014 22:34:08 -0700 Subject: [PATCH 294/641] Converted bang to ask to avoid scary warning when a block is removed Removing a block through the blockmanager gave a scary warning messages in the driver. ``` 2014-05-08 20:16:19,172 WARN BlockManagerMasterActor: Got unknown message: true 2014-05-08 20:16:19,172 WARN BlockManagerMasterActor: Got unknown message: true 2014-05-08 20:16:19,172 WARN BlockManagerMasterActor: Got unknown message: true ``` This is because the [BlockManagerSlaveActor](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala#L44) would send back an acknowledgement ("true"). But the BlockManagerMasterActor would have sent the RemoveBlock message as a send, not as ask(), so would reject the receiver "true" as a unknown message. @pwendell Author: Tathagata Das Closes #708 from tdas/bm-fix and squashes the following commits: ed4ef15 [Tathagata Das] Converted bang to ask to avoid scary warning when a block is removed. --- .../org/apache/spark/storage/BlockManagerMasterActor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 98fa0df6ec289..6aed322eeb185 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -250,7 +250,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus // Remove the block from the slave's BlockManager. // Doesn't actually wait for a confirmation and the message might get lost. // If message loss becomes frequent, we should add retry logic here. - blockManager.get.slaveActor ! RemoveBlock(blockId) + blockManager.get.slaveActor.ask(RemoveBlock(blockId))(akkaTimeout) } } } From bd67551ee724fd7cce4f2e2977a862216c992ef5 Mon Sep 17 00:00:00 2001 From: witgo Date: Fri, 9 May 2014 01:51:26 -0700 Subject: [PATCH 295/641] [SPARK-1760]: fix building spark with maven documentation Author: witgo Closes #712 from witgo/building-with-maven and squashes the following commits: 215523b [witgo] fix building spark with maven documentation --- docs/building-with-maven.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index cac01ded60d94..b6dd553bbe06b 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -96,7 +96,7 @@ Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.o The ScalaTest plugin also supports running only a specific test suite as follows: - $ mvn -Dhadoop.version=... -Dsuites=org.apache.spark.repl.ReplSuite test + $ mvn -Dhadoop.version=... -DwildcardSuites=org.apache.spark.repl.ReplSuite test ## Continuous Compilation ## From 59577df14c06417676a9ffdd599f5713c448e299 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Fri, 9 May 2014 14:51:34 -0700 Subject: [PATCH 296/641] SPARK-1770: Revert accidental(?) fix Looks like this change was accidentally committed here: https://github.com/apache/spark/commit/06b15baab25951d124bbe6b64906f4139e037deb but the change does not show up in the PR itself (#704). Other than not intending to go in with that PR, this also broke the test JavaAPISuite.repartition. Author: Aaron Davidson Closes #716 from aarondav/shufflerand and squashes the following commits: b1cf70b [Aaron Davidson] SPARK-1770: Revert accidental(?) fix --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 9d8d8044f07eb..a1ca612cc9a09 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -330,9 +330,9 @@ abstract class RDD[T: ClassTag]( if (shuffle) { // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( - new ShuffledRDD[Int, T, (Int, T)](map(x => (Utils.random.nextInt(), x)), + new ShuffledRDD[T, Null, (T, Null)](map(x => (x, null)), new HashPartitioner(numPartitions)), - numPartitions).values + numPartitions).keys } else { new CoalescedRDD(this, numPartitions) } From 2f452cbaf35dbc609ab48ec0ee5e3dd7b6b9b790 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Fri, 9 May 2014 21:50:23 -0700 Subject: [PATCH 297/641] SPARK-1686: keep schedule() calling in the main thread https://issues.apache.org/jira/browse/SPARK-1686 moved from original JIRA (by @markhamstra): In deploy.master.Master, the completeRecovery method is the last thing to be called when a standalone Master is recovering from failure. It is responsible for resetting some state, relaunching drivers, and eventually resuming its scheduling duties. There are currently four places in Master.scala where completeRecovery is called. Three of them are from within the actor's receive method, and aren't problems. The last starts from within receive when the ElectedLeader message is received, but the actual completeRecovery() call is made from the Akka scheduler. That means that it will execute on a different scheduler thread, and Master itself will end up running (i.e., schedule() ) from that Akka scheduler thread. In this PR, I added a new master message TriggerSchedule to trigger the "local" call of schedule() in the scheduler thread Author: CodingCat Closes #639 from CodingCat/SPARK-1686 and squashes the following commits: 81bb4ca [CodingCat] rename variable 69e0a2a [CodingCat] style fix 36a2ac0 [CodingCat] address Aaron's comments ec9b7bb [CodingCat] address the comments 02b37ca [CodingCat] keep schedule() calling in the main thread --- .../org/apache/spark/deploy/master/Master.scala | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index fdb633bd33608..f254f5585ba25 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -104,6 +104,8 @@ private[spark] class Master( var leaderElectionAgent: ActorRef = _ + private var recoveryCompletionTask: Cancellable = _ + // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. @@ -152,6 +154,10 @@ private[spark] class Master( } override def postStop() { + // prevent the CompleteRecovery message sending to restarted master + if (recoveryCompletionTask != null) { + recoveryCompletionTask.cancel() + } webUi.stop() fileSystemsUsed.foreach(_.close()) masterMetricsSystem.stop() @@ -171,10 +177,13 @@ private[spark] class Master( logInfo("I have been elected leader! New state: " + state) if (state == RecoveryState.RECOVERING) { beginRecovery(storedApps, storedDrivers, storedWorkers) - context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis) { completeRecovery() } + recoveryCompletionTask = context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis, self, + CompleteRecovery) } } + case CompleteRecovery => completeRecovery() + case RevokedLeadership => { logError("Leadership has been revoked -- master shutting down.") System.exit(0) @@ -465,7 +474,7 @@ private[spark] class Master( * Schedule the currently available resources among waiting apps. This method will be called * every time a new app joins or resource availability changes. */ - def schedule() { + private def schedule() { if (state != RecoveryState.ALIVE) { return } // First schedule drivers, they take strict precedence over applications @@ -485,7 +494,7 @@ private[spark] class Master( // Try to spread out each app among all the nodes, until it has all its cores for (app <- waitingApps if app.coresLeft > 0) { val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE) - .filter(canUse(app, _)).sortBy(_.coresFree).reverse + .filter(canUse(app, _)).sortBy(_.coresFree).reverse val numUsable = usableWorkers.length val assigned = new Array[Int](numUsable) // Number of cores to give on each node var toAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum) From 561510867a1b79beef57acf9df65c9f88481435d Mon Sep 17 00:00:00 2001 From: witgo Date: Sat, 10 May 2014 10:15:04 -0700 Subject: [PATCH 298/641] [SPARK-1644] The org.datanucleus:* should not be packaged into spark-assembly-*.jar Author: witgo Closes #688 from witgo/SPARK-1644 and squashes the following commits: 56ad6ac [witgo] review commit 87c03e4 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1644 6ffa7e4 [witgo] review commit a597414 [witgo] The org.datanucleus:* should not be packaged into spark-assembly-*.jar --- assembly/pom.xml | 1 + project/SparkBuild.scala | 11 ++++++----- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 7d123fb1d7f02..6c4d46aeb67bd 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -96,6 +96,7 @@ *:* + org.datanucleus:* META-INF/*.SF META-INF/*.DSA META-INF/*.RSA diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 7f9746ec4acc0..27e9505ec9831 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -579,12 +579,13 @@ object SparkBuild extends Build { def extraAssemblySettings() = Seq( test in assembly := {}, mergeStrategy in assembly := { - case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard - case m if m.toLowerCase.matches("meta-inf.*\\.sf$") => MergeStrategy.discard - case "log4j.properties" => MergeStrategy.discard + case PathList("org", "datanucleus", xs @ _*) => MergeStrategy.discard + case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard + case m if m.toLowerCase.matches("meta-inf.*\\.sf$") => MergeStrategy.discard + case "log4j.properties" => MergeStrategy.discard case m if m.toLowerCase.startsWith("meta-inf/services/") => MergeStrategy.filterDistinctLines - case "reference.conf" => MergeStrategy.concat - case _ => MergeStrategy.first + case "reference.conf" => MergeStrategy.concat + case _ => MergeStrategy.first } ) From 4d6055329846f5e09472e5f844127a5ab5880e15 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 10 May 2014 11:48:01 -0700 Subject: [PATCH 299/641] [SQL] Upgrade parquet library. I think we are hitting this issue in some perf tests: https://github.com/Parquet/parquet-mr/commit/6aed5288fd4a1398063a5a219b2ae4a9f71b02cf Credit to @aarondav ! Author: Michael Armbrust Closes #684 from marmbrus/upgradeParquet and squashes the following commits: e10a619 [Michael Armbrust] Upgrade parquet library. --- pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index fb904e868cfaf..e0bff60a54cde 100644 --- a/pom.xml +++ b/pom.xml @@ -123,7 +123,7 @@ ${hadoop.version} 0.94.6 0.12.0 - 1.3.2 + 1.4.3 1.2.3 8.1.14.v20131031 0.3.6 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 27e9505ec9831..af882b3ea7beb 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -300,7 +300,7 @@ object SparkBuild extends Build { val jets3tVersion = if ("^2\\.[3-9]+".r.findFirstIn(hadoopVersion).isDefined) "0.9.0" else "0.7.1" val jettyVersion = "8.1.14.v20131031" val hiveVersion = "0.12.0" - val parquetVersion = "1.3.2" + val parquetVersion = "1.4.3" val slf4jVersion = "1.7.5" val excludeNetty = ExclusionRule(organization = "org.jboss.netty") From 8e94d2721a9d3d36697e13f8cc6567ae8aeee78b Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Sat, 10 May 2014 12:03:27 -0700 Subject: [PATCH 300/641] [SPARK-1778] [SQL] Add 'limit' transformation to SchemaRDD. Add `limit` transformation to `SchemaRDD`. Author: Takuya UESHIN Closes #711 from ueshin/issues/SPARK-1778 and squashes the following commits: 33169df [Takuya UESHIN] Add 'limit' transformation to SchemaRDD. --- .../src/main/scala/org/apache/spark/sql/SchemaRDD.scala | 9 +++++++++ .../test/scala/org/apache/spark/sql/DslQuerySuite.scala | 6 ++++++ 2 files changed, 15 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 34200be3ac955..2569815ebb209 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -178,6 +178,15 @@ class SchemaRDD( def orderBy(sortExprs: SortOrder*): SchemaRDD = new SchemaRDD(sqlContext, Sort(sortExprs, logicalPlan)) + /** + * Limits the results by the given expressions. + * {{{ + * schemaRDD.limit(10) + * }}} + */ + def limit(limitExpr: Expression): SchemaRDD = + new SchemaRDD(sqlContext, Limit(limitExpr, logicalPlan)) + /** * Performs a grouping followed by an aggregation. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index be0f4a4c73b36..92a707ea57504 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -71,6 +71,12 @@ class DslQuerySuite extends QueryTest { Seq((3,1), (3,2), (2,1), (2,2), (1,1), (1,2))) } + test("limit") { + checkAnswer( + testData.limit(10), + testData.take(10).toSeq) + } + test("average") { checkAnswer( testData2.groupBy()(Average('a)), From 7eefc9d2b3f6ebc0ecb5562da7323f1e06afbb35 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 10 May 2014 12:10:24 -0700 Subject: [PATCH 301/641] SPARK-1708. Add a ClassTag on Serializer and things that depend on it This pull request contains a rebased patch from @heathermiller (https://github.com/heathermiller/spark/pull/1) to add ClassTags on Serializer and types that depend on it (Broadcast and AccumulableCollection). Putting these in the public API signatures now will allow us to use Scala Pickling for serialization down the line without breaking binary compatibility. One question remaining is whether we also want them on Accumulator -- Accumulator is passed as part of a bigger Task or TaskResult object via the closure serializer so it doesn't seem super useful to add the ClassTag there. Broadcast and AccumulableCollection in contrast were being serialized directly. CC @rxin, @pwendell, @heathermiller Author: Matei Zaharia Closes #700 from mateiz/spark-1708 and squashes the following commits: 1a3d8b0 [Matei Zaharia] Use fake ClassTag in Java 3b449ed [Matei Zaharia] test fix 2209a27 [Matei Zaharia] Code style fixes 9d48830 [Matei Zaharia] Add a ClassTag on Serializer and things that depend on it --- .../scala/org/apache/spark/Accumulators.scala | 7 +-- .../scala/org/apache/spark/SparkContext.scala | 4 +- .../spark/api/java/JavaSparkContext.scala | 2 +- .../apache/spark/broadcast/Broadcast.scala | 4 +- .../spark/broadcast/BroadcastFactory.scala | 4 +- .../spark/broadcast/BroadcastManager.scala | 4 +- .../spark/broadcast/HttpBroadcast.scala | 7 ++- .../broadcast/HttpBroadcastFactory.scala | 4 +- .../spark/broadcast/TorrentBroadcast.scala | 4 +- .../broadcast/TorrentBroadcastFactory.scala | 4 +- .../org/apache/spark/rdd/CheckpointRDD.scala | 4 +- .../spark/rdd/ParallelCollectionRDD.scala | 2 +- .../apache/spark/rdd/RDDCheckpointData.scala | 2 +- .../spark/serializer/JavaSerializer.scala | 13 +++--- .../spark/serializer/KryoSerializer.scala | 12 ++--- .../apache/spark/serializer/Serializer.scala | 17 +++---- .../scala/org/apache/spark/util/Utils.scala | 2 +- .../serializer/KryoSerializerSuite.scala | 11 ++--- .../bagel/WikipediaPageRankStandalone.scala | 12 ++--- .../spark/graphx/impl/Serializers.scala | 45 ++++++++++--------- .../apache/spark/graphx/SerializerSuite.scala | 5 ++- .../sql/execution/SparkSqlSerializer.scala | 6 ++- 22 files changed, 103 insertions(+), 72 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 6d652faae149a..cdfd338081fa2 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -21,6 +21,7 @@ import java.io.{ObjectInputStream, Serializable} import scala.collection.generic.Growable import scala.collection.mutable.Map +import scala.reflect.ClassTag import org.apache.spark.serializer.JavaSerializer @@ -164,9 +165,9 @@ trait AccumulableParam[R, T] extends Serializable { def zero(initialValue: R): R } -private[spark] -class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializable, T] - extends AccumulableParam[R,T] { +private[spark] class +GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] + extends AccumulableParam[R, T] { def addAccumulator(growable: R, elem: T): R = { growable += elem diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 9d7c2c8d3d630..c639b3e15ded5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -756,7 +756,7 @@ class SparkContext(config: SparkConf) extends Logging { * Growable and TraversableOnce are the standard APIs that guarantee += and ++=, implemented by * standard mutable collections. So you can use this with mutable Map, Set, etc. */ - def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable, T] + def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] (initialValue: R): Accumulable[R, T] = { val param = new GrowableAccumulableParam[R,T] new Accumulable(initialValue, param) @@ -767,7 +767,7 @@ class SparkContext(config: SparkConf) extends Logging { * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. * The variable will be sent to each cluster only once. */ - def broadcast[T](value: T): Broadcast[T] = { + def broadcast[T: ClassTag](value: T): Broadcast[T] = { val bc = env.broadcastManager.newBroadcast[T](value, isLocal) cleaner.foreach(_.registerBroadcastForCleanup(bc)) bc diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 8b95cda511643..a7cfee6d01711 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -447,7 +447,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. * The variable will be sent to each cluster only once. */ - def broadcast[T](value: T): Broadcast[T] = sc.broadcast(value) + def broadcast[T](value: T): Broadcast[T] = sc.broadcast(value)(fakeClassTag) /** Shut down the SparkContext. */ def stop() { diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index 738a3b1bed7f3..76956f6a345d1 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -21,6 +21,8 @@ import java.io.Serializable import org.apache.spark.SparkException +import scala.reflect.ClassTag + /** * A broadcast variable. Broadcast variables allow the programmer to keep a read-only variable * cached on each machine rather than shipping a copy of it with tasks. They can be used, for @@ -50,7 +52,7 @@ import org.apache.spark.SparkException * @param id A unique identifier for the broadcast variable. * @tparam T Type of the data contained in the broadcast variable. */ -abstract class Broadcast[T](val id: Long) extends Serializable { +abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable { /** * Flag signifying whether the broadcast variable is valid diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index 8c8ce9b1691ac..a8c827030a1ef 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -17,6 +17,8 @@ package org.apache.spark.broadcast +import scala.reflect.ClassTag + import org.apache.spark.SecurityManager import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi @@ -31,7 +33,7 @@ import org.apache.spark.annotation.DeveloperApi @DeveloperApi trait BroadcastFactory { def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit - def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] + def newBroadcast[T: ClassTag](value: T, isLocal: Boolean, id: Long): Broadcast[T] def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean): Unit def stop(): Unit } diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala index cf62aca4d45e8..c88be6aba6901 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala @@ -19,6 +19,8 @@ package org.apache.spark.broadcast import java.util.concurrent.atomic.AtomicLong +import scala.reflect.ClassTag + import org.apache.spark._ private[spark] class BroadcastManager( @@ -56,7 +58,7 @@ private[spark] class BroadcastManager( private val nextBroadcastId = new AtomicLong(0) - def newBroadcast[T](value_ : T, isLocal: Boolean) = { + def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean) = { broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement()) } diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 29372f16f2cac..78fc286e5192c 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -22,6 +22,8 @@ import java.io.{BufferedInputStream, BufferedOutputStream} import java.net.{URL, URLConnection, URI} import java.util.concurrent.TimeUnit +import scala.reflect.ClassTag + import org.apache.spark.{HttpServer, Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} @@ -34,7 +36,8 @@ import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedH * (through a HTTP server running at the driver) and stored in the BlockManager of the * executor to speed up future accesses. */ -private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) +private[spark] class HttpBroadcast[T: ClassTag]( + @transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) with Logging with Serializable { def getValue = value_ @@ -173,7 +176,7 @@ private[spark] object HttpBroadcast extends Logging { files += file.getAbsolutePath } - def read[T](id: Long): T = { + def read[T: ClassTag](id: Long): T = { logDebug("broadcast read server: " + serverUri + " id: broadcast-" + id) val url = serverUri + "/" + BroadcastBlockId(id).name diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala index e3f6cdc6154dd..d5a031e2bbb59 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala @@ -17,6 +17,8 @@ package org.apache.spark.broadcast +import scala.reflect.ClassTag + import org.apache.spark.{SecurityManager, SparkConf} /** @@ -29,7 +31,7 @@ class HttpBroadcastFactory extends BroadcastFactory { HttpBroadcast.initialize(isDriver, conf, securityMgr) } - def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = + def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = new HttpBroadcast[T](value_, isLocal, id) def stop() { HttpBroadcast.stop() } diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 2659274c5e98e..734de37ba115d 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -19,6 +19,7 @@ package org.apache.spark.broadcast import java.io.{ByteArrayInputStream, ObjectInputStream, ObjectOutputStream} +import scala.reflect.ClassTag import scala.math import scala.util.Random @@ -44,7 +45,8 @@ import org.apache.spark.util.Utils * copies of the broadcast data (one per executor) as done by the * [[org.apache.spark.broadcast.HttpBroadcast]]. */ -private[spark] class TorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) +private[spark] class TorrentBroadcast[T: ClassTag]( + @transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) with Logging with Serializable { def getValue = value_ diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala index d216b58718148..1de8396a0e17f 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala @@ -17,6 +17,8 @@ package org.apache.spark.broadcast +import scala.reflect.ClassTag + import org.apache.spark.{SecurityManager, SparkConf} /** @@ -30,7 +32,7 @@ class TorrentBroadcastFactory extends BroadcastFactory { TorrentBroadcast.initialize(isDriver, conf) } - def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = + def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = new TorrentBroadcast[T](value_, isLocal, id) def stop() { TorrentBroadcast.stop() } diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 888af541cf970..34c51b833025e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -84,7 +84,7 @@ private[spark] object CheckpointRDD extends Logging { "part-%05d".format(splitId) } - def writeToFile[T]( + def writeToFile[T: ClassTag]( path: String, broadcastedConf: Broadcast[SerializableWritable[Configuration]], blockSize: Int = -1 @@ -160,7 +160,7 @@ private[spark] object CheckpointRDD extends Logging { val conf = SparkHadoopUtil.get.newConfiguration() val fs = path.getFileSystem(conf) val broadcastedConf = sc.broadcast(new SerializableWritable(conf)) - sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, broadcastedConf, 1024) _) + sc.runJob(rdd, CheckpointRDD.writeToFile[Int](path.toString, broadcastedConf, 1024) _) val cpRDD = new CheckpointRDD[Int](sc, path.toString) assert(cpRDD.partitions.length == rdd.partitions.length, "Number of partitions is not the same") assert(cpRDD.collect.toList == rdd.collect.toList, "Data of partitions not the same") diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index 5f03d7d650a30..2425929fc73c5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -77,7 +77,7 @@ private[spark] class ParallelCollectionPartition[T: ClassTag]( slice = in.readInt() val ser = sfactory.newInstance() - Utils.deserializeViaNestedStream(in, ser)(ds => values = ds.readObject()) + Utils.deserializeViaNestedStream(in, ser)(ds => values = ds.readObject[Seq[T]]()) } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index 953f0555e57c5..c3b2a33fb54d0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -92,7 +92,7 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) // Save to file, and reload it as an RDD val broadcastedConf = rdd.context.broadcast( new SerializableWritable(rdd.context.hadoopConfiguration)) - rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path.toString, broadcastedConf) _) + rdd.context.runJob(rdd, CheckpointRDD.writeToFile[T](path.toString, broadcastedConf) _) val newRDD = new CheckpointRDD[T](rdd.context, path.toString) if (newRDD.partitions.size != rdd.partitions.size) { throw new SparkException( diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index e9163deaf2036..0a7e1ec539679 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -20,6 +20,8 @@ package org.apache.spark.serializer import java.io._ import java.nio.ByteBuffer +import scala.reflect.ClassTag + import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.ByteBufferInputStream @@ -36,7 +38,7 @@ private[spark] class JavaSerializationStream(out: OutputStream, counterReset: In * But only call it every 10,000th time to avoid bloated serialization streams (when * the stream 'resets' object class descriptions have to be re-written) */ - def writeObject[T](t: T): SerializationStream = { + def writeObject[T: ClassTag](t: T): SerializationStream = { objOut.writeObject(t) if (counterReset > 0 && counter >= counterReset) { objOut.reset() @@ -46,6 +48,7 @@ private[spark] class JavaSerializationStream(out: OutputStream, counterReset: In } this } + def flush() { objOut.flush() } def close() { objOut.close() } } @@ -57,12 +60,12 @@ extends DeserializationStream { Class.forName(desc.getName, false, loader) } - def readObject[T](): T = objIn.readObject().asInstanceOf[T] + def readObject[T: ClassTag](): T = objIn.readObject().asInstanceOf[T] def close() { objIn.close() } } private[spark] class JavaSerializerInstance(counterReset: Int) extends SerializerInstance { - def serialize[T](t: T): ByteBuffer = { + def serialize[T: ClassTag](t: T): ByteBuffer = { val bos = new ByteArrayOutputStream() val out = serializeStream(bos) out.writeObject(t) @@ -70,13 +73,13 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize ByteBuffer.wrap(bos.toByteArray) } - def deserialize[T](bytes: ByteBuffer): T = { + def deserialize[T: ClassTag](bytes: ByteBuffer): T = { val bis = new ByteBufferInputStream(bytes) val in = deserializeStream(bis) in.readObject().asInstanceOf[T] } - def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T = { + def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { val bis = new ByteBufferInputStream(bytes) val in = deserializeStream(bis, loader) in.readObject().asInstanceOf[T] diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index c4daec7875d26..5286f7b4c211a 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -31,6 +31,8 @@ import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage._ import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} +import scala.reflect.ClassTag + /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. * @@ -95,7 +97,7 @@ private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { val output = new KryoOutput(outStream) - def writeObject[T](t: T): SerializationStream = { + def writeObject[T: ClassTag](t: T): SerializationStream = { kryo.writeClassAndObject(output, t) this } @@ -108,7 +110,7 @@ private[spark] class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { val input = new KryoInput(inStream) - def readObject[T](): T = { + def readObject[T: ClassTag](): T = { try { kryo.readClassAndObject(input).asInstanceOf[T] } catch { @@ -131,18 +133,18 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ lazy val output = ks.newKryoOutput() lazy val input = new KryoInput() - def serialize[T](t: T): ByteBuffer = { + def serialize[T: ClassTag](t: T): ByteBuffer = { output.clear() kryo.writeClassAndObject(output, t) ByteBuffer.wrap(output.toBytes) } - def deserialize[T](bytes: ByteBuffer): T = { + def deserialize[T: ClassTag](bytes: ByteBuffer): T = { input.setBuffer(bytes.array) kryo.readClassAndObject(input).asInstanceOf[T] } - def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T = { + def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { val oldClassLoader = kryo.getClassLoader kryo.setClassLoader(loader) input.setBuffer(bytes.array) diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index f2c8f9b6218d6..ee26970a3d874 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -20,6 +20,8 @@ package org.apache.spark.serializer import java.io.{ByteArrayOutputStream, EOFException, InputStream, OutputStream} import java.nio.ByteBuffer +import scala.reflect.ClassTag + import org.apache.spark.SparkEnv import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.{ByteBufferInputStream, NextIterator} @@ -59,17 +61,17 @@ object Serializer { */ @DeveloperApi trait SerializerInstance { - def serialize[T](t: T): ByteBuffer + def serialize[T: ClassTag](t: T): ByteBuffer - def deserialize[T](bytes: ByteBuffer): T + def deserialize[T: ClassTag](bytes: ByteBuffer): T - def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T + def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T def serializeStream(s: OutputStream): SerializationStream def deserializeStream(s: InputStream): DeserializationStream - def serializeMany[T](iterator: Iterator[T]): ByteBuffer = { + def serializeMany[T: ClassTag](iterator: Iterator[T]): ByteBuffer = { // Default implementation uses serializeStream val stream = new ByteArrayOutputStream() serializeStream(stream).writeAll(iterator) @@ -85,18 +87,17 @@ trait SerializerInstance { } } - /** * :: DeveloperApi :: * A stream for writing serialized objects. */ @DeveloperApi trait SerializationStream { - def writeObject[T](t: T): SerializationStream + def writeObject[T: ClassTag](t: T): SerializationStream def flush(): Unit def close(): Unit - def writeAll[T](iter: Iterator[T]): SerializationStream = { + def writeAll[T: ClassTag](iter: Iterator[T]): SerializationStream = { while (iter.hasNext) { writeObject(iter.next()) } @@ -111,7 +112,7 @@ trait SerializationStream { */ @DeveloperApi trait DeserializationStream { - def readObject[T](): T + def readObject[T: ClassTag](): T def close(): Unit /** diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 3f0ed61c5bbfb..95777fbf57d8b 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -850,7 +850,7 @@ private[spark] object Utils extends Logging { /** * Clone an object using a Spark serializer. */ - def clone[T](value: T, serializer: SerializerInstance): T = { + def clone[T: ClassTag](value: T, serializer: SerializerInstance): T = { serializer.deserialize[T](serializer.serialize(value)) } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 5d4673aebe9e8..cdd6b3d8feed7 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.serializer import scala.collection.mutable +import scala.reflect.ClassTag import com.esotericsoftware.kryo.Kryo import org.scalatest.FunSuite @@ -31,7 +32,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("basic types") { val ser = new KryoSerializer(conf).newInstance() - def check[T](t: T) { + def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } check(1) @@ -61,7 +62,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("pairs") { val ser = new KryoSerializer(conf).newInstance() - def check[T](t: T) { + def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } check((1, 1)) @@ -85,7 +86,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("Scala data structures") { val ser = new KryoSerializer(conf).newInstance() - def check[T](t: T) { + def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } check(List[Int]()) @@ -108,7 +109,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("ranges") { val ser = new KryoSerializer(conf).newInstance() - def check[T](t: T) { + def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) // Check that very long ranges don't get written one element at a time assert(ser.serialize(t).limit < 100) @@ -129,7 +130,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("custom registrator") { val ser = new KryoSerializer(conf).newInstance() - def check[T](t: T) { + def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index a197dac87d6db..576a3e371b993 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -28,6 +28,8 @@ import org.apache.spark.serializer.{DeserializationStream, SerializationStream, import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag + object WikipediaPageRankStandalone { def main(args: Array[String]) { if (args.length < 4) { @@ -143,15 +145,15 @@ class WPRSerializer extends org.apache.spark.serializer.Serializer { } class WPRSerializerInstance extends SerializerInstance { - def serialize[T](t: T): ByteBuffer = { + def serialize[T: ClassTag](t: T): ByteBuffer = { throw new UnsupportedOperationException() } - def deserialize[T](bytes: ByteBuffer): T = { + def deserialize[T: ClassTag](bytes: ByteBuffer): T = { throw new UnsupportedOperationException() } - def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T = { + def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { throw new UnsupportedOperationException() } @@ -167,7 +169,7 @@ class WPRSerializerInstance extends SerializerInstance { class WPRSerializationStream(os: OutputStream) extends SerializationStream { val dos = new DataOutputStream(os) - def writeObject[T](t: T): SerializationStream = t match { + def writeObject[T: ClassTag](t: T): SerializationStream = t match { case (id: String, wrapper: ArrayBuffer[_]) => wrapper(0) match { case links: Array[String] => { dos.writeInt(0) // links @@ -200,7 +202,7 @@ class WPRSerializationStream(os: OutputStream) extends SerializationStream { class WPRDeserializationStream(is: InputStream) extends DeserializationStream { val dis = new DataInputStream(is) - def readObject[T](): T = { + def readObject[T: ClassTag](): T = { val typeId = dis.readInt() typeId match { case 0 => { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala index 2f0531ee5f379..1de42eeca1f00 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -17,20 +17,22 @@ package org.apache.spark.graphx.impl +import scala.language.existentials + import java.io.{EOFException, InputStream, OutputStream} import java.nio.ByteBuffer +import scala.reflect.ClassTag + import org.apache.spark.graphx._ import org.apache.spark.serializer._ -import scala.language.existentials - private[graphx] class VertexIdMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T](t: T) = { + def writeObject[T: ClassTag](t: T) = { val msg = t.asInstanceOf[(VertexId, _)] writeVarLong(msg._1, optimizePositive = false) this @@ -38,7 +40,7 @@ class VertexIdMsgSerializer extends Serializer with Serializable { } override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T](): T = { + override def readObject[T: ClassTag](): T = { (readVarLong(optimizePositive = false), null).asInstanceOf[T] } } @@ -51,7 +53,7 @@ class IntVertexBroadcastMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T](t: T) = { + def writeObject[T: ClassTag](t: T) = { val msg = t.asInstanceOf[VertexBroadcastMsg[Int]] writeVarLong(msg.vid, optimizePositive = false) writeInt(msg.data) @@ -60,7 +62,7 @@ class IntVertexBroadcastMsgSerializer extends Serializer with Serializable { } override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T](): T = { + override def readObject[T: ClassTag](): T = { val a = readVarLong(optimizePositive = false) val b = readInt() new VertexBroadcastMsg[Int](0, a, b).asInstanceOf[T] @@ -75,7 +77,7 @@ class LongVertexBroadcastMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T](t: T) = { + def writeObject[T: ClassTag](t: T) = { val msg = t.asInstanceOf[VertexBroadcastMsg[Long]] writeVarLong(msg.vid, optimizePositive = false) writeLong(msg.data) @@ -84,7 +86,7 @@ class LongVertexBroadcastMsgSerializer extends Serializer with Serializable { } override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T](): T = { + override def readObject[T: ClassTag](): T = { val a = readVarLong(optimizePositive = false) val b = readLong() new VertexBroadcastMsg[Long](0, a, b).asInstanceOf[T] @@ -99,7 +101,7 @@ class DoubleVertexBroadcastMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T](t: T) = { + def writeObject[T: ClassTag](t: T) = { val msg = t.asInstanceOf[VertexBroadcastMsg[Double]] writeVarLong(msg.vid, optimizePositive = false) writeDouble(msg.data) @@ -108,7 +110,7 @@ class DoubleVertexBroadcastMsgSerializer extends Serializer with Serializable { } override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - def readObject[T](): T = { + def readObject[T: ClassTag](): T = { val a = readVarLong(optimizePositive = false) val b = readDouble() new VertexBroadcastMsg[Double](0, a, b).asInstanceOf[T] @@ -123,7 +125,7 @@ class IntAggMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T](t: T) = { + def writeObject[T: ClassTag](t: T) = { val msg = t.asInstanceOf[(VertexId, Int)] writeVarLong(msg._1, optimizePositive = false) writeUnsignedVarInt(msg._2) @@ -132,7 +134,7 @@ class IntAggMsgSerializer extends Serializer with Serializable { } override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T](): T = { + override def readObject[T: ClassTag](): T = { val a = readVarLong(optimizePositive = false) val b = readUnsignedVarInt() (a, b).asInstanceOf[T] @@ -147,7 +149,7 @@ class LongAggMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T](t: T) = { + def writeObject[T: ClassTag](t: T) = { val msg = t.asInstanceOf[(VertexId, Long)] writeVarLong(msg._1, optimizePositive = false) writeVarLong(msg._2, optimizePositive = true) @@ -156,7 +158,7 @@ class LongAggMsgSerializer extends Serializer with Serializable { } override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T](): T = { + override def readObject[T: ClassTag](): T = { val a = readVarLong(optimizePositive = false) val b = readVarLong(optimizePositive = true) (a, b).asInstanceOf[T] @@ -171,7 +173,7 @@ class DoubleAggMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T](t: T) = { + def writeObject[T: ClassTag](t: T) = { val msg = t.asInstanceOf[(VertexId, Double)] writeVarLong(msg._1, optimizePositive = false) writeDouble(msg._2) @@ -180,7 +182,7 @@ class DoubleAggMsgSerializer extends Serializer with Serializable { } override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - def readObject[T](): T = { + def readObject[T: ClassTag](): T = { val a = readVarLong(optimizePositive = false) val b = readDouble() (a, b).asInstanceOf[T] @@ -196,7 +198,7 @@ class DoubleAggMsgSerializer extends Serializer with Serializable { private[graphx] abstract class ShuffleSerializationStream(s: OutputStream) extends SerializationStream { // The implementation should override this one. - def writeObject[T](t: T): SerializationStream + def writeObject[T: ClassTag](t: T): SerializationStream def writeInt(v: Int) { s.write(v >> 24) @@ -309,7 +311,7 @@ abstract class ShuffleSerializationStream(s: OutputStream) extends Serialization private[graphx] abstract class ShuffleDeserializationStream(s: InputStream) extends DeserializationStream { // The implementation should override this one. - def readObject[T](): T + def readObject[T: ClassTag](): T def readInt(): Int = { val first = s.read() @@ -398,11 +400,12 @@ abstract class ShuffleDeserializationStream(s: InputStream) extends Deserializat private[graphx] sealed trait ShuffleSerializerInstance extends SerializerInstance { - override def serialize[T](t: T): ByteBuffer = throw new UnsupportedOperationException + override def serialize[T: ClassTag](t: T): ByteBuffer = throw new UnsupportedOperationException - override def deserialize[T](bytes: ByteBuffer): T = throw new UnsupportedOperationException + override def deserialize[T: ClassTag](bytes: ByteBuffer): T = + throw new UnsupportedOperationException - override def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T = + override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = throw new UnsupportedOperationException // The implementation should override the following two. diff --git a/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala index 73438d9535962..91caa6b605a1e 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.graphx import java.io.{EOFException, ByteArrayInputStream, ByteArrayOutputStream} import scala.util.Random +import scala.reflect.ClassTag import org.scalatest.FunSuite @@ -164,7 +165,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { def testVarLongEncoding(v: Long, optimizePositive: Boolean) { val bout = new ByteArrayOutputStream val stream = new ShuffleSerializationStream(bout) { - def writeObject[T](t: T): SerializationStream = { + def writeObject[T: ClassTag](t: T): SerializationStream = { writeVarLong(t.asInstanceOf[Long], optimizePositive = optimizePositive) this } @@ -173,7 +174,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { val bin = new ByteArrayInputStream(bout.toByteArray) val dstream = new ShuffleDeserializationStream(bin) { - def readObject[T](): T = { + def readObject[T: ClassTag](): T = { readVarLong(optimizePositive).asInstanceOf[T] } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 5067c14ddffeb..1c6e29b3cdee9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution import java.nio.ByteBuffer +import scala.reflect.ClassTag + import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Serializer, Kryo} @@ -59,11 +61,11 @@ private[sql] object SparkSqlSerializer { new KryoSerializer(sparkConf) } - def serialize[T](o: T): Array[Byte] = { + def serialize[T: ClassTag](o: T): Array[Byte] = { ser.newInstance().serialize(o).array() } - def deserialize[T](bytes: Array[Byte]): T = { + def deserialize[T: ClassTag](bytes: Array[Byte]): T = { ser.newInstance().deserialize[T](ByteBuffer.wrap(bytes)) } } From c05d11bb307eaba40c5669da2d374c28debaa55a Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Sat, 10 May 2014 12:46:51 -0700 Subject: [PATCH 302/641] fix broken in link in python docs Author: Andy Konwinski Closes #650 from andyk/python-docs-link-fix and squashes the following commits: a1f9d51 [Andy Konwinski] fix broken in link in python docs --- docs/python-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 6813963bb080c..39fb5f0c99ca3 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -45,7 +45,7 @@ errors = logData.filter(is_error) PySpark will automatically ship these functions to executors, along with any objects that they reference. Instances of classes will be serialized and shipped to executors by PySpark, but classes themselves cannot be automatically distributed to executors. -The [Standalone Use](#standalone-use) section describes how to ship code dependencies to executors. +The [Standalone Use](#standalone-programs) section describes how to ship code dependencies to executors. In addition, PySpark fully supports interactive use---simply run `./bin/pyspark` to launch an interactive shell. From 3776f2f283842543ff766398292532c6e94221cc Mon Sep 17 00:00:00 2001 From: Bouke van der Bijl Date: Sat, 10 May 2014 13:02:13 -0700 Subject: [PATCH 303/641] Add Python includes to path before depickling broadcast values This fixes https://issues.apache.org/jira/browse/SPARK-1731 by adding the Python includes to the PYTHONPATH before depickling the broadcast values @airhorns Author: Bouke van der Bijl Closes #656 from bouk/python-includes-before-broadcast and squashes the following commits: 7b0dfe4 [Bouke van der Bijl] Add Python includes to path before depickling broadcast values --- .../org/apache/spark/api/python/PythonRDD.scala | 10 +++++----- python/pyspark/worker.py | 14 +++++++------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index fecd9762f3f60..388b838d78bba 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -179,6 +179,11 @@ private[spark] class PythonRDD[T: ClassTag]( dataOut.writeInt(split.index) // sparkFilesDir PythonRDD.writeUTF(SparkFiles.getRootDirectory, dataOut) + // Python includes (*.zip and *.egg files) + dataOut.writeInt(pythonIncludes.length) + for (include <- pythonIncludes) { + PythonRDD.writeUTF(include, dataOut) + } // Broadcast variables dataOut.writeInt(broadcastVars.length) for (broadcast <- broadcastVars) { @@ -186,11 +191,6 @@ private[spark] class PythonRDD[T: ClassTag]( dataOut.writeInt(broadcast.value.length) dataOut.write(broadcast.value) } - // Python includes (*.zip and *.egg files) - dataOut.writeInt(pythonIncludes.length) - for (include <- pythonIncludes) { - PythonRDD.writeUTF(include, dataOut) - } dataOut.flush() // Serialized command: dataOut.writeInt(command.length) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 4c214ef359685..f43210c6c0301 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -56,13 +56,6 @@ def main(infile, outfile): SparkFiles._root_directory = spark_files_dir SparkFiles._is_running_on_worker = True - # fetch names and values of broadcast variables - num_broadcast_variables = read_int(infile) - for _ in range(num_broadcast_variables): - bid = read_long(infile) - value = pickleSer._read_with_length(infile) - _broadcastRegistry[bid] = Broadcast(bid, value) - # fetch names of includes (*.zip and *.egg files) and construct PYTHONPATH sys.path.append(spark_files_dir) # *.py files that were added will be copied here num_python_includes = read_int(infile) @@ -70,6 +63,13 @@ def main(infile, outfile): filename = utf8_deserializer.loads(infile) sys.path.append(os.path.join(spark_files_dir, filename)) + # fetch names and values of broadcast variables + num_broadcast_variables = read_int(infile) + for _ in range(num_broadcast_variables): + bid = read_long(infile) + value = pickleSer._read_with_length(infile) + _broadcastRegistry[bid] = Broadcast(bid, value) + command = pickleSer._read_with_length(infile) (func, deserializer, serializer) = command init_time = time.time() From 6c2691d0a0ed46a8b8093e05a4708706cf187168 Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Sat, 10 May 2014 14:01:08 -0700 Subject: [PATCH 304/641] [SPARK-1690] Tolerating empty elements when saving Python RDD to text files Tolerate empty strings in PythonRDD Author: Kan Zhang Closes #644 from kanzhang/SPARK-1690 and squashes the following commits: c62ad33 [Kan Zhang] Adding Python doctest 473ec4b [Kan Zhang] [SPARK-1690] Tolerating empty elements when saving Python RDD to text files --- .../scala/org/apache/spark/api/python/PythonRDD.scala | 5 +++-- python/pyspark/rdd.py | 8 ++++++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 388b838d78bba..2971c277aa863 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -94,6 +94,7 @@ private[spark] class PythonRDD[T: ClassTag]( val obj = new Array[Byte](length) stream.readFully(obj) obj + case 0 => Array.empty[Byte] case SpecialLengths.TIMING_DATA => // Timing data from worker val bootTime = stream.readLong() @@ -123,7 +124,7 @@ private[spark] class PythonRDD[T: ClassTag]( stream.readFully(update) accumulator += Collections.singletonList(update) } - Array.empty[Byte] + null } } catch { @@ -143,7 +144,7 @@ private[spark] class PythonRDD[T: ClassTag]( var _nextObj = read() - def hasNext = _nextObj.length != 0 + def hasNext = _nextObj != null } new InterruptibleIterator(context, stdoutIterator) } diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 3a1c56af5b221..4f74824ba4cf2 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -891,6 +891,14 @@ def saveAsTextFile(self, path): >>> from glob import glob >>> ''.join(sorted(input(glob(tempFile.name + "/part-0000*")))) '0\\n1\\n2\\n3\\n4\\n5\\n6\\n7\\n8\\n9\\n' + + Empty lines are tolerated when saving to text files. + + >>> tempFile2 = NamedTemporaryFile(delete=True) + >>> tempFile2.close() + >>> sc.parallelize(['', 'foo', '', 'bar', '']).saveAsTextFile(tempFile2.name) + >>> ''.join(sorted(input(glob(tempFile2.name + "/part-0000*")))) + '\\n\\n\\nbar\\nfoo\\n' """ def func(split, iterator): for x in iterator: From 905173df57b90f90ebafb22e43f55164445330e6 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 10 May 2014 14:48:07 -0700 Subject: [PATCH 305/641] Unify GraphImpl RDDs + other graph load optimizations This PR makes the following changes, primarily in e4fbd329aef85fe2c38b0167255d2a712893d683: 1. *Unify RDDs to avoid zipPartitions.* A graph used to be four RDDs: vertices, edges, routing table, and triplet view. This commit merges them down to two: vertices (with routing table), and edges (with replicated vertices). 2. *Avoid duplicate shuffle in graph building.* We used to do two shuffles when building a graph: one to extract routing information from the edges and move it to the vertices, and another to find nonexistent vertices referred to by edges. With this commit, the latter is done as a side effect of the former. 3. *Avoid no-op shuffle when joins are fully eliminated.* This is a side effect of unifying the edges and the triplet view. 4. *Join elimination for mapTriplets.* 5. *Ship only the needed vertex attributes when upgrading the triplet view.* If the triplet view already contains source attributes, and we now need both attributes, only ship destination attributes rather than re-shipping both. This is done in `ReplicatedVertexView#upgrade`. Author: Ankur Dave Closes #497 from ankurdave/unify-rdds and squashes the following commits: 332ab43 [Ankur Dave] Merge remote-tracking branch 'apache-spark/master' into unify-rdds 4933e2e [Ankur Dave] Exclude RoutingTable from binary compatibility check 5ba8789 [Ankur Dave] Add GraphX upgrade guide from Spark 0.9.1 13ac845 [Ankur Dave] Merge remote-tracking branch 'apache-spark/master' into unify-rdds a04765c [Ankur Dave] Remove unnecessary toOps call 57202e8 [Ankur Dave] Replace case with pair parameter 75af062 [Ankur Dave] Add explicit return types 04d3ae5 [Ankur Dave] Convert implicit parameter to context bound c88b269 [Ankur Dave] Revert upgradeIterator to if-in-a-loop 0d3584c [Ankur Dave] EdgePartition.size should be val 2a928b2 [Ankur Dave] Set locality wait 10b3596 [Ankur Dave] Clean up public API ae36110 [Ankur Dave] Fix style errors e4fbd32 [Ankur Dave] Unify GraphImpl RDDs + other graph load optimizations d6d60e2 [Ankur Dave] In GraphLoader, coalesce to minEdgePartitions 62c7b78 [Ankur Dave] In Analytics, take PageRank numIter d64e8d4 [Ankur Dave] Log current Pregel iteration --- docs/graphx-programming-guide.md | 22 +- .../org/apache/spark/graphx/EdgeRDD.scala | 56 +-- .../org/apache/spark/graphx/EdgeTriplet.scala | 2 + .../scala/org/apache/spark/graphx/Graph.scala | 2 +- .../spark/graphx/GraphKryoRegistrator.scala | 8 +- .../org/apache/spark/graphx/GraphLoader.scala | 10 +- .../org/apache/spark/graphx/GraphOps.scala | 17 +- .../org/apache/spark/graphx/Pregel.scala | 6 +- .../org/apache/spark/graphx/VertexRDD.scala | 166 ++++++--- .../spark/graphx/impl/EdgePartition.scala | 132 +++++-- .../graphx/impl/EdgePartitionBuilder.scala | 18 +- .../graphx/impl/EdgeTripletIterator.scala | 50 ++- .../apache/spark/graphx/impl/GraphImpl.scala | 344 +++++++----------- .../graphx/impl/MessageToPartition.scala | 21 +- .../graphx/impl/ReplicatedVertexView.scala | 238 ++++-------- .../spark/graphx/impl/RoutingTable.scala | 82 ----- .../graphx/impl/RoutingTablePartition.scala | 158 ++++++++ .../spark/graphx/impl/Serializers.scala | 29 ++ .../impl/ShippableVertexPartition.scala | 149 ++++++++ .../spark/graphx/impl/VertexPartition.scala | 269 ++------------ .../graphx/impl/VertexPartitionBase.scala | 91 +++++ .../graphx/impl/VertexPartitionBaseOps.scala | 245 +++++++++++++ .../apache/spark/graphx/lib/Analytics.scala | 8 +- .../org/apache/spark/graphx/GraphSuite.scala | 10 +- .../graphx/impl/EdgePartitionSuite.scala | 48 ++- .../impl/EdgeTripletIteratorSuite.scala | 10 +- .../graphx/impl/VertexPartitionSuite.scala | 11 - project/MimaBuild.scala | 2 + 28 files changed, 1353 insertions(+), 851 deletions(-) delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 07be8ba58efa3..42ab27bf55ccf 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -86,6 +86,12 @@ support the [Bagel API](api/scala/index.html#org.apache.spark.bagel.package) and [Bagel programming guide](bagel-programming-guide.html). However, we encourage Bagel users to explore the new GraphX API and comment on issues that may complicate the transition from Bagel. +## Upgrade Guide from Spark 0.9.1 + +GraphX in Spark {{site.SPARK_VERSION}} contains one user-facing interface change from Spark 0.9.1. [`EdgeRDD`][EdgeRDD] may now store adjacent vertex attributes to construct the triplets, so it has gained a type parameter. The edges of a graph of type `Graph[VD, ED]` are of type `EdgeRDD[ED, VD]` rather than `EdgeRDD[ED]`. + +[EdgeRDD]: api/scala/index.html#org.apache.spark.graphx.EdgeRDD + # Getting Started To get started you first need to import Spark and GraphX into your project, as follows: @@ -145,12 +151,12 @@ the vertices and edges of the graph: {% highlight scala %} class Graph[VD, ED] { val vertices: VertexRDD[VD] - val edges: EdgeRDD[ED] + val edges: EdgeRDD[ED, VD] } {% endhighlight %} -The classes `VertexRDD[VD]` and `EdgeRDD[ED]` extend and are optimized versions of `RDD[(VertexID, -VD)]` and `RDD[Edge[ED]]` respectively. Both `VertexRDD[VD]` and `EdgeRDD[ED]` provide additional +The classes `VertexRDD[VD]` and `EdgeRDD[ED, VD]` extend and are optimized versions of `RDD[(VertexID, +VD)]` and `RDD[Edge[ED]]` respectively. Both `VertexRDD[VD]` and `EdgeRDD[ED, VD]` provide additional functionality built around graph computation and leverage internal optimizations. We discuss the `VertexRDD` and `EdgeRDD` API in greater detail in the section on [vertex and edge RDDs](#vertex_and_edge_rdds) but for now they can be thought of as simply RDDs of the form: @@ -302,7 +308,7 @@ class Graph[VD, ED] { val degrees: VertexRDD[Int] // Views of the graph as collections ============================================================= val vertices: VertexRDD[VD] - val edges: EdgeRDD[ED] + val edges: EdgeRDD[ED, VD] val triplets: RDD[EdgeTriplet[VD, ED]] // Functions for caching graphs ================================================================== def persist(newLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED] @@ -908,7 +914,7 @@ val setC: VertexRDD[Double] = setA.innerJoin(setB)((id, a, b) => a + b) ## EdgeRDDs -The `EdgeRDD[ED]`, which extends `RDD[Edge[ED]]` organizes the edges in blocks partitioned using one +The `EdgeRDD[ED, VD]`, which extends `RDD[Edge[ED]]` organizes the edges in blocks partitioned using one of the various partitioning strategies defined in [`PartitionStrategy`][PartitionStrategy]. Within each partition, edge attributes and adjacency structure, are stored separately enabling maximum reuse when changing attribute values. @@ -918,11 +924,11 @@ reuse when changing attribute values. The three additional functions exposed by the `EdgeRDD` are: {% highlight scala %} // Transform the edge attributes while preserving the structure -def mapValues[ED2](f: Edge[ED] => ED2): EdgeRDD[ED2] +def mapValues[ED2](f: Edge[ED] => ED2): EdgeRDD[ED2, VD] // Revere the edges reusing both attributes and structure -def reverse: EdgeRDD[ED] +def reverse: EdgeRDD[ED, VD] // Join two `EdgeRDD`s partitioned using the same partitioning strategy. -def innerJoin[ED2, ED3](other: EdgeRDD[ED2])(f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3] +def innerJoin[ED2, ED3](other: EdgeRDD[ED2, VD])(f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3, VD] {% endhighlight %} In most applications we have found that operations on the `EdgeRDD` are accomplished through the diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index fa78ca99b8891..a8fc095072512 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -20,16 +20,19 @@ package org.apache.spark.graphx import scala.reflect.{classTag, ClassTag} import org.apache.spark.{OneToOneDependency, Partition, Partitioner, TaskContext} -import org.apache.spark.graphx.impl.EdgePartition import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.graphx.impl.EdgePartition + /** - * `EdgeRDD[ED]` extends `RDD[Edge[ED]]` by storing the edges in columnar format on each partition - * for performance. + * `EdgeRDD[ED, VD]` extends `RDD[Edge[ED]]` by storing the edges in columnar format on each + * partition for performance. It may additionally store the vertex attributes associated with each + * edge to provide the triplet view. Shipping of the vertex attributes is managed by + * `impl.ReplicatedVertexView`. */ -class EdgeRDD[@specialized ED: ClassTag]( - val partitionsRDD: RDD[(PartitionID, EdgePartition[ED])]) +class EdgeRDD[@specialized ED: ClassTag, VD: ClassTag]( + val partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])]) extends RDD[Edge[ED]](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { partitionsRDD.setName("EdgeRDD") @@ -45,8 +48,12 @@ class EdgeRDD[@specialized ED: ClassTag]( partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD))) override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = { - val p = firstParent[(PartitionID, EdgePartition[ED])].iterator(part, context) - p.next._2.iterator.map(_.copy()) + val p = firstParent[(PartitionID, EdgePartition[ED, VD])].iterator(part, context) + if (p.hasNext) { + p.next._2.iterator.map(_.copy()) + } else { + Iterator.empty + } } override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect() @@ -61,11 +68,15 @@ class EdgeRDD[@specialized ED: ClassTag]( this } - private[graphx] def mapEdgePartitions[ED2: ClassTag]( - f: (PartitionID, EdgePartition[ED]) => EdgePartition[ED2]): EdgeRDD[ED2] = { - new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => - val (pid, ep) = iter.next() - Iterator(Tuple2(pid, f(pid, ep))) + private[graphx] def mapEdgePartitions[ED2: ClassTag, VD2: ClassTag]( + f: (PartitionID, EdgePartition[ED, VD]) => EdgePartition[ED2, VD2]): EdgeRDD[ED2, VD2] = { + new EdgeRDD[ED2, VD2](partitionsRDD.mapPartitions({ iter => + if (iter.hasNext) { + val (pid, ep) = iter.next() + Iterator(Tuple2(pid, f(pid, ep))) + } else { + Iterator.empty + } }, preservesPartitioning = true)) } @@ -76,7 +87,7 @@ class EdgeRDD[@specialized ED: ClassTag]( * @param f the function from an edge to a new edge value * @return a new EdgeRDD containing the new edge values */ - def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDD[ED2] = + def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDD[ED2, VD] = mapEdgePartitions((pid, part) => part.map(f)) /** @@ -84,7 +95,14 @@ class EdgeRDD[@specialized ED: ClassTag]( * * @return a new EdgeRDD containing all the edges reversed */ - def reverse: EdgeRDD[ED] = mapEdgePartitions((pid, part) => part.reverse) + def reverse: EdgeRDD[ED, VD] = mapEdgePartitions((pid, part) => part.reverse) + + /** Removes all edges but those matching `epred` and where both vertices match `vpred`. */ + def filter( + epred: EdgeTriplet[VD, ED] => Boolean, + vpred: (VertexId, VD) => Boolean): EdgeRDD[ED, VD] = { + mapEdgePartitions((pid, part) => part.filter(epred, vpred)) + } /** * Inner joins this EdgeRDD with another EdgeRDD, assuming both are partitioned using the same @@ -96,19 +114,15 @@ class EdgeRDD[@specialized ED: ClassTag]( * with values supplied by `f` */ def innerJoin[ED2: ClassTag, ED3: ClassTag] - (other: EdgeRDD[ED2]) - (f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3] = { + (other: EdgeRDD[ED2, _]) + (f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3, VD] = { val ed2Tag = classTag[ED2] val ed3Tag = classTag[ED3] - new EdgeRDD[ED3](partitionsRDD.zipPartitions(other.partitionsRDD, true) { + new EdgeRDD[ED3, VD](partitionsRDD.zipPartitions(other.partitionsRDD, true) { (thisIter, otherIter) => val (pid, thisEPart) = thisIter.next() val (_, otherEPart) = otherIter.next() Iterator(Tuple2(pid, thisEPart.innerJoin(otherEPart)(f)(ed2Tag, ed3Tag))) }) } - - private[graphx] def collectVertexIds(): RDD[VertexId] = { - partitionsRDD.flatMap { case (_, p) => Array.concat(p.srcIds, p.dstIds) } - } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala index dfc6a801587d2..9d473d5ebda44 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala @@ -63,4 +63,6 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { if (srcId == vid) srcAttr else { assert(dstId == vid); dstAttr } override def toString = ((srcId, srcAttr), (dstId, dstAttr), attr).toString() + + def toTuple: ((VertexId, VD), (VertexId, VD), ED) = ((srcId, srcAttr), (dstId, dstAttr), attr) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 50395868902dc..dc5dac4fdad57 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -59,7 +59,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * along with their vertex data. * */ - @transient val edges: EdgeRDD[ED] + @transient val edges: EdgeRDD[ED, VD] /** * An RDD containing the edge triplets, which are edges along with the vertex data associated with diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index dd380d8c182c9..d295d0127ac72 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -19,10 +19,11 @@ package org.apache.spark.graphx import com.esotericsoftware.kryo.Kryo -import org.apache.spark.graphx.impl._ import org.apache.spark.serializer.KryoRegistrator -import org.apache.spark.util.collection.BitSet import org.apache.spark.util.BoundedPriorityQueue +import org.apache.spark.util.collection.BitSet + +import org.apache.spark.graphx.impl._ /** * Registers GraphX classes with Kryo for improved performance. @@ -33,8 +34,9 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[Edge[Object]]) kryo.register(classOf[MessageToPartition[Object]]) kryo.register(classOf[VertexBroadcastMsg[Object]]) + kryo.register(classOf[RoutingTableMessage]) kryo.register(classOf[(VertexId, Object)]) - kryo.register(classOf[EdgePartition[Object]]) + kryo.register(classOf[EdgePartition[Object, Object]]) kryo.register(classOf[BitSet]) kryo.register(classOf[VertexIdToIndexMap]) kryo.register(classOf[VertexAttributeBlock[Object]]) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala index 18858466db27b..389490c139848 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -47,8 +47,7 @@ object GraphLoader extends Logging { * @param path the path to the file (e.g., /home/data/file or hdfs://file) * @param canonicalOrientation whether to orient edges in the positive * direction - * @param minEdgePartitions the number of partitions for the - * the edge RDD + * @param minEdgePartitions the number of partitions for the edge RDD */ def edgeListFile( sc: SparkContext, @@ -60,8 +59,9 @@ object GraphLoader extends Logging { val startTime = System.currentTimeMillis // Parse the edge data table directly into edge partitions - val edges = sc.textFile(path, minEdgePartitions).mapPartitionsWithIndex { (pid, iter) => - val builder = new EdgePartitionBuilder[Int] + val lines = sc.textFile(path, minEdgePartitions).coalesce(minEdgePartitions) + val edges = lines.mapPartitionsWithIndex { (pid, iter) => + val builder = new EdgePartitionBuilder[Int, Int] iter.foreach { line => if (!line.isEmpty && line(0) != '#') { val lineArray = line.split("\\s+") @@ -78,7 +78,7 @@ object GraphLoader extends Logging { } } Iterator((pid, builder.toEdgePartition)) - }.cache() + }.cache().setName("GraphLoader.edgeListFile - edges (%s)".format(path)) edges.count() logInfo("It took %d ms to load the edges".format(System.currentTimeMillis - startTime)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 4997fbc3cbcd8..edd5b79da1522 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -18,11 +18,13 @@ package org.apache.spark.graphx import scala.reflect.ClassTag -import org.apache.spark.SparkContext._ +import scala.util.Random + import org.apache.spark.SparkException -import org.apache.spark.graphx.lib._ +import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD -import scala.util.Random + +import org.apache.spark.graphx.lib._ /** * Contains additional functionality for [[Graph]]. All operations are expressed in terms of the @@ -43,19 +45,22 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * The in-degree of each vertex in the graph. * @note Vertices with no in-edges are not returned in the resulting RDD. */ - @transient lazy val inDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.In) + @transient lazy val inDegrees: VertexRDD[Int] = + degreesRDD(EdgeDirection.In).setName("GraphOps.inDegrees") /** * The out-degree of each vertex in the graph. * @note Vertices with no out-edges are not returned in the resulting RDD. */ - @transient lazy val outDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Out) + @transient lazy val outDegrees: VertexRDD[Int] = + degreesRDD(EdgeDirection.Out).setName("GraphOps.outDegrees") /** * The degree of each vertex in the graph. * @note Vertices with no edges are not returned in the resulting RDD. */ - @transient lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Either) + @transient lazy val degrees: VertexRDD[Int] = + degreesRDD(EdgeDirection.Either).setName("GraphOps.degrees") /** * Computes the neighboring vertex degrees. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala index ac07a594a12e4..4572eab2875bb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -18,6 +18,7 @@ package org.apache.spark.graphx import scala.reflect.ClassTag +import org.apache.spark.Logging /** @@ -52,7 +53,7 @@ import scala.reflect.ClassTag * }}} * */ -object Pregel { +object Pregel extends Logging { /** * Execute a Pregel-like iterative vertex-parallel abstraction. The @@ -142,6 +143,9 @@ object Pregel { // hides oldMessages (depended on by newVerts), newVerts (depended on by messages), and the // vertices of prevG (depended on by newVerts, oldMessages, and the vertices of g). activeMessages = messages.count() + + logInfo("Pregel finished iteration " + i) + // Unpersist the RDDs hidden by newly-materialized RDDs oldMessages.unpersist(blocking=false) newVerts.unpersist(blocking=false) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index f0fc605c88575..8c62897037b6d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -24,8 +24,11 @@ import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ import org.apache.spark.storage.StorageLevel -import org.apache.spark.graphx.impl.MsgRDDFunctions -import org.apache.spark.graphx.impl.VertexPartition +import org.apache.spark.graphx.impl.RoutingTablePartition +import org.apache.spark.graphx.impl.ShippableVertexPartition +import org.apache.spark.graphx.impl.VertexAttributeBlock +import org.apache.spark.graphx.impl.RoutingTableMessageRDDFunctions._ +import org.apache.spark.graphx.impl.VertexRDDFunctions._ /** * Extends `RDD[(VertexId, VD)]` by ensuring that there is only one entry for each vertex and by @@ -33,6 +36,9 @@ import org.apache.spark.graphx.impl.VertexPartition * joined efficiently. All operations except [[reindex]] preserve the index. To construct a * `VertexRDD`, use the [[org.apache.spark.graphx.VertexRDD$ VertexRDD object]]. * + * Additionally, stores routing information to enable joining the vertex attributes with an + * [[EdgeRDD]]. + * * @example Construct a `VertexRDD` from a plain RDD: * {{{ * // Construct an initial vertex set @@ -50,13 +56,11 @@ import org.apache.spark.graphx.impl.VertexPartition * @tparam VD the vertex attribute associated with each vertex in the set. */ class VertexRDD[@specialized VD: ClassTag]( - val partitionsRDD: RDD[VertexPartition[VD]]) + val partitionsRDD: RDD[ShippableVertexPartition[VD]]) extends RDD[(VertexId, VD)](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { require(partitionsRDD.partitioner.isDefined) - partitionsRDD.setName("VertexRDD") - /** * Construct a new VertexRDD that is indexed by only the visible vertices. The resulting * VertexRDD will be based on a different index and can no longer be quickly joined with this @@ -71,6 +75,16 @@ class VertexRDD[@specialized VD: ClassTag]( override protected def getPreferredLocations(s: Partition): Seq[String] = partitionsRDD.preferredLocations(s) + override def setName(_name: String): this.type = { + if (partitionsRDD.name != null) { + partitionsRDD.setName(partitionsRDD.name + ", " + _name) + } else { + partitionsRDD.setName(_name) + } + this + } + setName("VertexRDD") + override def persist(newLevel: StorageLevel): this.type = { partitionsRDD.persist(newLevel) this @@ -90,14 +104,14 @@ class VertexRDD[@specialized VD: ClassTag]( * Provides the `RDD[(VertexId, VD)]` equivalent output. */ override def compute(part: Partition, context: TaskContext): Iterator[(VertexId, VD)] = { - firstParent[VertexPartition[VD]].iterator(part, context).next.iterator + firstParent[ShippableVertexPartition[VD]].iterator(part, context).next.iterator } /** * Applies a function to each `VertexPartition` of this RDD and returns a new VertexRDD. */ private[graphx] def mapVertexPartitions[VD2: ClassTag]( - f: VertexPartition[VD] => VertexPartition[VD2]) + f: ShippableVertexPartition[VD] => ShippableVertexPartition[VD2]) : VertexRDD[VD2] = { val newPartitionsRDD = partitionsRDD.mapPartitions(_.map(f), preservesPartitioning = true) new VertexRDD(newPartitionsRDD) @@ -208,10 +222,8 @@ class VertexRDD[@specialized VD: ClassTag]( case _ => new VertexRDD[VD3]( partitionsRDD.zipPartitions( - other.partitionBy(this.partitioner.get), preservesPartitioning = true) - { (part, msgs) => - val vertexPartition: VertexPartition[VD] = part.next() - Iterator(vertexPartition.leftJoin(msgs)(f)) + other.copartitionWithVertices(this.partitioner.get), preservesPartitioning = true) { + (partIter, msgs) => partIter.map(_.leftJoin(msgs)(f)) } ) } @@ -254,10 +266,8 @@ class VertexRDD[@specialized VD: ClassTag]( case _ => new VertexRDD( partitionsRDD.zipPartitions( - other.partitionBy(this.partitioner.get), preservesPartitioning = true) - { (part, msgs) => - val vertexPartition: VertexPartition[VD] = part.next() - Iterator(vertexPartition.innerJoin(msgs)(f)) + other.copartitionWithVertices(this.partitioner.get), preservesPartitioning = true) { + (partIter, msgs) => partIter.map(_.innerJoin(msgs)(f)) } ) } @@ -276,14 +286,31 @@ class VertexRDD[@specialized VD: ClassTag]( */ def aggregateUsingIndex[VD2: ClassTag]( messages: RDD[(VertexId, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { - val shuffled = MsgRDDFunctions.partitionForAggregation(messages, this.partitioner.get) + val shuffled = messages.copartitionWithVertices(this.partitioner.get) val parts = partitionsRDD.zipPartitions(shuffled, true) { (thisIter, msgIter) => - val vertexPartition: VertexPartition[VD] = thisIter.next() - Iterator(vertexPartition.aggregateUsingIndex(msgIter, reduceFunc)) + thisIter.map(_.aggregateUsingIndex(msgIter, reduceFunc)) } new VertexRDD[VD2](parts) } + /** + * Returns a new `VertexRDD` reflecting a reversal of all edge directions in the corresponding + * [[EdgeRDD]]. + */ + def reverseRoutingTables(): VertexRDD[VD] = + this.mapVertexPartitions(vPart => vPart.withRoutingTable(vPart.routingTable.reverse)) + + /** Generates an RDD of vertex attributes suitable for shipping to the edge partitions. */ + private[graphx] def shipVertexAttributes( + shipSrc: Boolean, shipDst: Boolean): RDD[(PartitionID, VertexAttributeBlock[VD])] = { + partitionsRDD.mapPartitions(_.flatMap(_.shipVertexAttributes(shipSrc, shipDst))) + } + + /** Generates an RDD of vertex IDs suitable for shipping to the edge partitions. */ + private[graphx] def shipVertexIds(): RDD[(PartitionID, Array[VertexId])] = { + partitionsRDD.mapPartitions(_.flatMap(_.shipVertexIds())) + } + } // end of VertexRDD @@ -293,52 +320,101 @@ class VertexRDD[@specialized VD: ClassTag]( object VertexRDD { /** - * Construct a `VertexRDD` from an RDD of vertex-attribute pairs. - * Duplicate entries are removed arbitrarily. + * Constructs a standalone `VertexRDD` (one that is not set up for efficient joins with an + * [[EdgeRDD]]) from an RDD of vertex-attribute pairs. Duplicate entries are removed arbitrarily. * * @tparam VD the vertex attribute type * - * @param rdd the collection of vertex-attribute pairs + * @param vertices the collection of vertex-attribute pairs */ - def apply[VD: ClassTag](rdd: RDD[(VertexId, VD)]): VertexRDD[VD] = { - val partitioned: RDD[(VertexId, VD)] = rdd.partitioner match { - case Some(p) => rdd - case None => rdd.partitionBy(new HashPartitioner(rdd.partitions.size)) + def apply[VD: ClassTag](vertices: RDD[(VertexId, VD)]): VertexRDD[VD] = { + val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match { + case Some(p) => vertices + case None => vertices.copartitionWithVertices(new HashPartitioner(vertices.partitions.size)) } - val vertexPartitions = partitioned.mapPartitions( - iter => Iterator(VertexPartition(iter)), + val vertexPartitions = vPartitioned.mapPartitions( + iter => Iterator(ShippableVertexPartition(iter)), preservesPartitioning = true) new VertexRDD(vertexPartitions) } /** - * Constructs a `VertexRDD` from an RDD of vertex-attribute pairs, merging duplicates using - * `mergeFunc`. + * Constructs a `VertexRDD` from an RDD of vertex-attribute pairs. Duplicate vertex entries are + * removed arbitrarily. The resulting `VertexRDD` will be joinable with `edges`, and any missing + * vertices referred to by `edges` will be created with the attribute `defaultVal`. * * @tparam VD the vertex attribute type * - * @param rdd the collection of vertex-attribute pairs - * @param mergeFunc the associative, commutative merge function. + * @param vertices the collection of vertex-attribute pairs + * @param edges the [[EdgeRDD]] that these vertices may be joined with + * @param defaultVal the vertex attribute to use when creating missing vertices */ - def apply[VD: ClassTag](rdd: RDD[(VertexId, VD)], mergeFunc: (VD, VD) => VD): VertexRDD[VD] = { - val partitioned: RDD[(VertexId, VD)] = rdd.partitioner match { - case Some(p) => rdd - case None => rdd.partitionBy(new HashPartitioner(rdd.partitions.size)) + def apply[VD: ClassTag]( + vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_, _], defaultVal: VD): VertexRDD[VD] = { + VertexRDD(vertices, edges, defaultVal, (a, b) => b) + } + + /** + * Constructs a `VertexRDD` from an RDD of vertex-attribute pairs. Duplicate vertex entries are + * merged using `mergeFunc`. The resulting `VertexRDD` will be joinable with `edges`, and any + * missing vertices referred to by `edges` will be created with the attribute `defaultVal`. + * + * @tparam VD the vertex attribute type + * + * @param vertices the collection of vertex-attribute pairs + * @param edges the [[EdgeRDD]] that these vertices may be joined with + * @param defaultVal the vertex attribute to use when creating missing vertices + * @param mergeFunc the commutative, associative duplicate vertex attribute merge function + */ + def apply[VD: ClassTag]( + vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_, _], defaultVal: VD, mergeFunc: (VD, VD) => VD + ): VertexRDD[VD] = { + val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match { + case Some(p) => vertices + case None => vertices.copartitionWithVertices(new HashPartitioner(vertices.partitions.size)) + } + val routingTables = createRoutingTables(edges, vPartitioned.partitioner.get) + val vertexPartitions = vPartitioned.zipPartitions(routingTables, preservesPartitioning = true) { + (vertexIter, routingTableIter) => + val routingTable = + if (routingTableIter.hasNext) routingTableIter.next() else RoutingTablePartition.empty + Iterator(ShippableVertexPartition(vertexIter, routingTable, defaultVal)) } - val vertexPartitions = partitioned.mapPartitions( - iter => Iterator(VertexPartition(iter)), - preservesPartitioning = true) new VertexRDD(vertexPartitions) } /** - * Constructs a VertexRDD from the vertex IDs in `vids`, taking attributes from `rdd` and using - * `defaultVal` otherwise. + * Constructs a `VertexRDD` containing all vertices referred to in `edges`. The vertices will be + * created with the attribute `defaultVal`. The resulting `VertexRDD` will be joinable with + * `edges`. + * + * @tparam VD the vertex attribute type + * + * @param edges the [[EdgeRDD]] referring to the vertices to create + * @param numPartitions the desired number of partitions for the resulting `VertexRDD` + * @param defaultVal the vertex attribute to use when creating missing vertices */ - def apply[VD: ClassTag](vids: RDD[VertexId], rdd: RDD[(VertexId, VD)], defaultVal: VD) - : VertexRDD[VD] = { - VertexRDD(vids.map(vid => (vid, defaultVal))).leftJoin(rdd) { (vid, default, value) => - value.getOrElse(default) - } + def fromEdges[VD: ClassTag]( + edges: EdgeRDD[_, _], numPartitions: Int, defaultVal: VD): VertexRDD[VD] = { + val routingTables = createRoutingTables(edges, new HashPartitioner(numPartitions)) + val vertexPartitions = routingTables.mapPartitions({ routingTableIter => + val routingTable = + if (routingTableIter.hasNext) routingTableIter.next() else RoutingTablePartition.empty + Iterator(ShippableVertexPartition(Iterator.empty, routingTable, defaultVal)) + }, preservesPartitioning = true) + new VertexRDD(vertexPartitions) + } + + private def createRoutingTables( + edges: EdgeRDD[_, _], vertexPartitioner: Partitioner): RDD[RoutingTablePartition] = { + // Determine which vertices each edge partition needs by creating a mapping from vid to pid. + val vid2pid = edges.partitionsRDD.mapPartitions(_.flatMap( + Function.tupled(RoutingTablePartition.edgePartitionToMsgs))) + .setName("VertexRDD.createRoutingTables - vid2pid (aggregation)") + + val numEdgePartitions = edges.partitions.size + vid2pid.copartitionWithVertices(vertexPartitioner).mapPartitions( + iter => Iterator(RoutingTablePartition.fromMsgs(numEdgePartitions, iter)), + preservesPartitioning = true) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index b7c472e905a9b..871e81f8d245c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -17,39 +17,86 @@ package org.apache.spark.graphx.impl -import scala.reflect.ClassTag +import scala.reflect.{classTag, ClassTag} import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** - * A collection of edges stored in 3 large columnar arrays (src, dst, attribute). The arrays are - * clustered by src. + * A collection of edges stored in columnar format, along with any vertex attributes referenced. The + * edges are stored in 3 large columnar arrays (src, dst, attribute). The arrays are clustered by + * src. There is an optional active vertex set for filtering computation on the edges. + * + * @tparam ED the edge attribute type + * @tparam VD the vertex attribute type * * @param srcIds the source vertex id of each edge * @param dstIds the destination vertex id of each edge * @param data the attribute associated with each edge * @param index a clustered index on source vertex id - * @tparam ED the edge attribute type. + * @param vertices a map from referenced vertex ids to their corresponding attributes. Must + * contain all vertex ids from `srcIds` and `dstIds`, though not necessarily valid attributes for + * those vertex ids. The mask is not used. + * @param activeSet an optional active vertex set for filtering computation on the edges */ private[graphx] -class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag]( +class EdgePartition[ + @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag, VD: ClassTag]( @transient val srcIds: Array[VertexId], @transient val dstIds: Array[VertexId], @transient val data: Array[ED], - @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int]) extends Serializable { + @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int], + @transient val vertices: VertexPartition[VD], + @transient val activeSet: Option[VertexSet] = None + ) extends Serializable { + + /** Return a new `EdgePartition` with the specified edge data. */ + def withData[ED2: ClassTag](data_ : Array[ED2]): EdgePartition[ED2, VD] = { + new EdgePartition(srcIds, dstIds, data_, index, vertices, activeSet) + } + + /** Return a new `EdgePartition` with the specified vertex partition. */ + def withVertices[VD2: ClassTag]( + vertices_ : VertexPartition[VD2]): EdgePartition[ED, VD2] = { + new EdgePartition(srcIds, dstIds, data, index, vertices_, activeSet) + } + + /** Return a new `EdgePartition` with the specified active set, provided as an iterator. */ + def withActiveSet(iter: Iterator[VertexId]): EdgePartition[ED, VD] = { + val newActiveSet = new VertexSet + iter.foreach(newActiveSet.add(_)) + new EdgePartition(srcIds, dstIds, data, index, vertices, Some(newActiveSet)) + } + + /** Return a new `EdgePartition` with the specified active set. */ + def withActiveSet(activeSet_ : Option[VertexSet]): EdgePartition[ED, VD] = { + new EdgePartition(srcIds, dstIds, data, index, vertices, activeSet_) + } + + /** Return a new `EdgePartition` with updates to vertex attributes specified in `iter`. */ + def updateVertices(iter: Iterator[(VertexId, VD)]): EdgePartition[ED, VD] = { + this.withVertices(vertices.innerJoinKeepLeft(iter)) + } + + /** Look up vid in activeSet, throwing an exception if it is None. */ + def isActive(vid: VertexId): Boolean = { + activeSet.get.contains(vid) + } + + /** The number of active vertices, if any exist. */ + def numActives: Option[Int] = activeSet.map(_.size) /** * Reverse all the edges in this partition. * * @return a new edge partition with all edges reversed. */ - def reverse: EdgePartition[ED] = { - val builder = new EdgePartitionBuilder(size) + def reverse: EdgePartition[ED, VD] = { + val builder = new EdgePartitionBuilder(size)(classTag[ED], classTag[VD]) for (e <- iterator) { builder.add(e.dstId, e.srcId, e.attr) } - builder.toEdgePartition + builder.toEdgePartition.withVertices(vertices).withActiveSet(activeSet) } /** @@ -64,7 +111,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * @return a new edge partition with the result of the function `f` * applied to each edge */ - def map[ED2: ClassTag](f: Edge[ED] => ED2): EdgePartition[ED2] = { + def map[ED2: ClassTag](f: Edge[ED] => ED2): EdgePartition[ED2, VD] = { val newData = new Array[ED2](data.size) val edge = new Edge[ED]() val size = data.size @@ -76,7 +123,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) newData(i) = f(edge) i += 1 } - new EdgePartition(srcIds, dstIds, newData, index) + this.withData(newData) } /** @@ -91,7 +138,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * @tparam ED2 the type of the new attribute * @return a new edge partition with the attribute values replaced */ - def map[ED2: ClassTag](iter: Iterator[ED2]): EdgePartition[ED2] = { + def map[ED2: ClassTag](iter: Iterator[ED2]): EdgePartition[ED2, VD] = { // Faster than iter.toArray, because the expected size is known. val newData = new Array[ED2](data.size) var i = 0 @@ -100,7 +147,23 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) i += 1 } assert(newData.size == i) - new EdgePartition(srcIds, dstIds, newData, index) + this.withData(newData) + } + + /** + * Construct a new edge partition containing only the edges matching `epred` and where both + * vertices match `vpred`. + */ + def filter( + epred: EdgeTriplet[VD, ED] => Boolean, + vpred: (VertexId, VD) => Boolean): EdgePartition[ED, VD] = { + val filtered = tripletIterator().filter(et => + vpred(et.srcId, et.srcAttr) && vpred(et.dstId, et.dstAttr) && epred(et)) + val builder = new EdgePartitionBuilder[ED, VD] + for (e <- filtered) { + builder.add(e.srcId, e.dstId, e.attr) + } + builder.toEdgePartition.withVertices(vertices).withActiveSet(activeSet) } /** @@ -119,8 +182,8 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * @param merge a commutative associative merge operation * @return a new edge partition without duplicate edges */ - def groupEdges(merge: (ED, ED) => ED): EdgePartition[ED] = { - val builder = new EdgePartitionBuilder[ED] + def groupEdges(merge: (ED, ED) => ED): EdgePartition[ED, VD] = { + val builder = new EdgePartitionBuilder[ED, VD] var currSrcId: VertexId = null.asInstanceOf[VertexId] var currDstId: VertexId = null.asInstanceOf[VertexId] var currAttr: ED = null.asInstanceOf[ED] @@ -141,11 +204,11 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) if (size > 0) { builder.add(currSrcId, currDstId, currAttr) } - builder.toEdgePartition + builder.toEdgePartition.withVertices(vertices).withActiveSet(activeSet) } /** - * Apply `f` to all edges present in both `this` and `other` and return a new EdgePartition + * Apply `f` to all edges present in both `this` and `other` and return a new `EdgePartition` * containing the resulting edges. * * If there are multiple edges with the same src and dst in `this`, `f` will be invoked once for @@ -155,9 +218,9 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * once. */ def innerJoin[ED2: ClassTag, ED3: ClassTag] - (other: EdgePartition[ED2]) - (f: (VertexId, VertexId, ED, ED2) => ED3): EdgePartition[ED3] = { - val builder = new EdgePartitionBuilder[ED3] + (other: EdgePartition[ED2, _]) + (f: (VertexId, VertexId, ED, ED2) => ED3): EdgePartition[ED3, VD] = { + val builder = new EdgePartitionBuilder[ED3, VD] var i = 0 var j = 0 // For i = index of each edge in `this`... @@ -175,7 +238,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) } i += 1 } - builder.toEdgePartition + builder.toEdgePartition.withVertices(vertices).withActiveSet(activeSet) } /** @@ -183,7 +246,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * * @return size of the partition */ - def size: Int = srcIds.size + val size: Int = srcIds.size /** The number of unique source vertices in the partition. */ def indexSize: Int = index.size @@ -211,10 +274,35 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) } } + /** + * Get an iterator over the edge triplets in this partition. + * + * It is safe to keep references to the objects from this iterator. + */ + def tripletIterator( + includeSrc: Boolean = true, includeDst: Boolean = true): Iterator[EdgeTriplet[VD, ED]] = { + new EdgeTripletIterator(this, includeSrc, includeDst) + } + + /** + * Upgrade the given edge iterator into a triplet iterator. + * + * Be careful not to keep references to the objects from this iterator. To improve GC performance + * the same object is re-used in `next()`. + */ + def upgradeIterator( + edgeIter: Iterator[Edge[ED]], includeSrc: Boolean = true, includeDst: Boolean = true) + : Iterator[EdgeTriplet[VD, ED]] = { + new ReusingEdgeTripletIterator(edgeIter, this, includeSrc, includeDst) + } + /** * Get an iterator over the edges in this partition whose source vertex ids match srcIdPred. The * iterator is generated using an index scan, so it is efficient at skipping edges that don't * match srcIdPred. + * + * Be careful not to keep references to the objects from this iterator. To improve GC performance + * the same object is re-used in `next()`. */ def indexIterator(srcIdPred: VertexId => Boolean): Iterator[Edge[ED]] = index.iterator.filter(kv => srcIdPred(kv._1)).flatMap(Function.tupled(clusterIterator)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index 63ccccb056b48..ecb49bef42e45 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -20,12 +20,14 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import scala.util.Sorting +import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveVector} + import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap -import org.apache.spark.util.collection.PrimitiveVector private[graphx] -class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: Int = 64) { +class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: ClassTag]( + size: Int = 64) { var edges = new PrimitiveVector[Edge[ED]](size) /** Add a new edge to the partition. */ @@ -33,7 +35,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: I edges += Edge(src, dst, d) } - def toEdgePartition: EdgePartition[ED] = { + def toEdgePartition: EdgePartition[ED, VD] = { val edgeArray = edges.trim().array Sorting.quickSort(edgeArray)(Edge.lexicographicOrdering) val srcIds = new Array[VertexId](edgeArray.size) @@ -57,6 +59,14 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: I i += 1 } } - new EdgePartition(srcIds, dstIds, data, index) + + // Create and populate a VertexPartition with vids from the edges, but no attributes + val vidsIter = srcIds.iterator ++ dstIds.iterator + val vertexIds = new OpenHashSet[VertexId] + vidsIter.foreach(vid => vertexIds.add(vid)) + val vertices = new VertexPartition( + vertexIds, new Array[VD](vertexIds.capacity), vertexIds.getBitSet) + + new EdgePartition(srcIds, dstIds, data, index, vertices) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index 220a89d73d711..ebb0b9418d65d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -23,32 +23,62 @@ import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** - * The Iterator type returned when constructing edge triplets. This class technically could be - * an anonymous class in GraphImpl.triplets, but we name it here explicitly so it is easier to - * debug / profile. + * The Iterator type returned when constructing edge triplets. This could be an anonymous class in + * EdgePartition.tripletIterator, but we name it here explicitly so it is easier to debug / profile. */ private[impl] class EdgeTripletIterator[VD: ClassTag, ED: ClassTag]( - val vidToIndex: VertexIdToIndexMap, - val vertexArray: Array[VD], - val edgePartition: EdgePartition[ED]) + val edgePartition: EdgePartition[ED, VD], + val includeSrc: Boolean, + val includeDst: Boolean) extends Iterator[EdgeTriplet[VD, ED]] { // Current position in the array. private var pos = 0 - private val vmap = new PrimitiveKeyOpenHashMap[VertexId, VD](vidToIndex, vertexArray) - override def hasNext: Boolean = pos < edgePartition.size override def next() = { val triplet = new EdgeTriplet[VD, ED] triplet.srcId = edgePartition.srcIds(pos) - triplet.srcAttr = vmap(triplet.srcId) + if (includeSrc) { + triplet.srcAttr = edgePartition.vertices(triplet.srcId) + } triplet.dstId = edgePartition.dstIds(pos) - triplet.dstAttr = vmap(triplet.dstId) + if (includeDst) { + triplet.dstAttr = edgePartition.vertices(triplet.dstId) + } triplet.attr = edgePartition.data(pos) pos += 1 triplet } } + +/** + * An Iterator type for internal use that reuses EdgeTriplet objects. This could be an anonymous + * class in EdgePartition.upgradeIterator, but we name it here explicitly so it is easier to debug / + * profile. + */ +private[impl] +class ReusingEdgeTripletIterator[VD: ClassTag, ED: ClassTag]( + val edgeIter: Iterator[Edge[ED]], + val edgePartition: EdgePartition[ED, VD], + val includeSrc: Boolean, + val includeDst: Boolean) + extends Iterator[EdgeTriplet[VD, ED]] { + + private val triplet = new EdgeTriplet[VD, ED] + + override def hasNext = edgeIter.hasNext + + override def next() = { + triplet.set(edgeIter.next()) + if (includeSrc) { + triplet.srcAttr = edgePartition.vertices(triplet.srcId) + } + if (includeDst) { + triplet.dstAttr = edgePartition.vertices(triplet.dstId) + } + triplet + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 9eabccdee48db..2f2d0e03fd7b5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -19,54 +19,45 @@ package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} -import org.apache.spark.util.collection.PrimitiveVector -import org.apache.spark.{HashPartitioner, Partitioner} +import org.apache.spark.HashPartitioner import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.{RDD, ShuffledRDD} +import org.apache.spark.storage.StorageLevel + import org.apache.spark.graphx._ import org.apache.spark.graphx.impl.GraphImpl._ import org.apache.spark.graphx.impl.MsgRDDFunctions._ import org.apache.spark.graphx.util.BytecodeUtils -import org.apache.spark.rdd.{ShuffledRDD, RDD} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.ClosureCleaner /** - * A graph that supports computation on graphs. + * An implementation of [[org.apache.spark.graphx.Graph]] to support computation on graphs. * - * Graphs are represented using two classes of data: vertex-partitioned and - * edge-partitioned. `vertices` contains vertex attributes, which are vertex-partitioned. `edges` - * contains edge attributes, which are edge-partitioned. For operations on vertex neighborhoods, - * vertex attributes are replicated to the edge partitions where they appear as sources or - * destinations. `routingTable` stores the routing information for shipping vertex attributes to - * edge partitions. `replicatedVertexView` stores a view of the replicated vertex attributes created - * using the routing table. + * Graphs are represented using two RDDs: `vertices`, which contains vertex attributes and the + * routing information for shipping vertex attributes to edge partitions, and + * `replicatedVertexView`, which contains edges and the vertex attributes mentioned by each edge. */ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( @transient val vertices: VertexRDD[VD], - @transient val edges: EdgeRDD[ED], - @transient val routingTable: RoutingTable, - @transient val replicatedVertexView: ReplicatedVertexView[VD]) + @transient val replicatedVertexView: ReplicatedVertexView[VD, ED]) extends Graph[VD, ED] with Serializable { /** Default constructor is provided to support serialization */ - protected def this() = this(null, null, null, null) + protected def this() = this(null, null) + + @transient override val edges: EdgeRDD[ED, VD] = replicatedVertexView.edges /** Return a RDD that brings edges together with their source and destination vertices. */ - @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { - val vdTag = classTag[VD] - val edTag = classTag[ED] - edges.partitionsRDD.zipPartitions( - replicatedVertexView.get(true, true), true) { (ePartIter, vPartIter) => - val (pid, ePart) = ePartIter.next() - val (_, vPart) = vPartIter.next() - new EdgeTripletIterator(vPart.index, vPart.values, ePart)(vdTag, edTag) - } + @transient override lazy val triplets: RDD[EdgeTriplet[VD, ED]] = { + replicatedVertexView.upgrade(vertices, true, true) + replicatedVertexView.edges.partitionsRDD.mapPartitions(_.flatMap { + case (pid, part) => part.tripletIterator() + }) } override def persist(newLevel: StorageLevel): Graph[VD, ED] = { vertices.persist(newLevel) - edges.persist(newLevel) + replicatedVertexView.edges.persist(newLevel) this } @@ -74,14 +65,15 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] = { vertices.unpersist(blocking) - replicatedVertexView.unpersist(blocking) + // TODO: unpersist the replicated vertices in `replicatedVertexView` but leave the edges alone this } override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { - val numPartitions = edges.partitions.size + val numPartitions = replicatedVertexView.edges.partitions.size val edTag = classTag[ED] - val newEdges = new EdgeRDD(edges.map { e => + val vdTag = classTag[VD] + val newEdges = new EdgeRDD(replicatedVertexView.edges.map { e => val part: PartitionID = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) // Should we be using 3-tuple or an optimized class @@ -89,105 +81,79 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } .partitionBy(new HashPartitioner(numPartitions)) .mapPartitionsWithIndex( { (pid, iter) => - val builder = new EdgePartitionBuilder[ED]()(edTag) + val builder = new EdgePartitionBuilder[ED, VD]()(edTag, vdTag) iter.foreach { message => val data = message.data builder.add(data._1, data._2, data._3) } val edgePartition = builder.toEdgePartition Iterator((pid, edgePartition)) - }, preservesPartitioning = true).cache()) - GraphImpl(vertices, newEdges) + }, preservesPartitioning = true)) + GraphImpl.fromExistingRDDs(vertices, newEdges) } override def reverse: Graph[VD, ED] = { - val newETable = edges.mapEdgePartitions((pid, part) => part.reverse) - GraphImpl(vertices, newETable) + new GraphImpl(vertices.reverseRoutingTables(), replicatedVertexView.reverse()) } override def mapVertices[VD2: ClassTag](f: (VertexId, VD) => VD2): Graph[VD2, ED] = { if (classTag[VD] equals classTag[VD2]) { + vertices.cache() // The map preserves type, so we can use incremental replication val newVerts = vertices.mapVertexPartitions(_.map(f)).cache() val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) - val newReplicatedVertexView = new ReplicatedVertexView[VD2]( - changedVerts, edges, routingTable, - Some(replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2]])) - new GraphImpl(newVerts, edges, routingTable, newReplicatedVertexView) + val newReplicatedVertexView = replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2, ED]] + .updateVertices(changedVerts) + new GraphImpl(newVerts, newReplicatedVertexView) } else { // The map does not preserve type, so we must re-replicate all vertices - GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, routingTable) + GraphImpl(vertices.mapVertexPartitions(_.map(f)), replicatedVertexView.edges) } } override def mapEdges[ED2: ClassTag]( f: (PartitionID, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] = { - val newETable = edges.mapEdgePartitions((pid, part) => part.map(f(pid, part.iterator))) - new GraphImpl(vertices, newETable , routingTable, replicatedVertexView) + val newEdges = replicatedVertexView.edges + .mapEdgePartitions((pid, part) => part.map(f(pid, part.iterator))) + new GraphImpl(vertices, replicatedVertexView.withEdges(newEdges)) } override def mapTriplets[ED2: ClassTag]( f: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] = { - val newEdgePartitions = - edges.partitionsRDD.zipPartitions(replicatedVertexView.get(true, true), true) { - (ePartIter, vTableReplicatedIter) => - val (ePid, edgePartition) = ePartIter.next() - val (vPid, vPart) = vTableReplicatedIter.next() - assert(!vTableReplicatedIter.hasNext) - assert(ePid == vPid) - val et = new EdgeTriplet[VD, ED] - val inputIterator = edgePartition.iterator.map { e => - et.set(e) - et.srcAttr = vPart(e.srcId) - et.dstAttr = vPart(e.dstId) - et - } - // Apply the user function to the vertex partition - val outputIter = f(ePid, inputIterator) - // Consume the iterator to update the edge attributes - val newEdgePartition = edgePartition.map(outputIter) - Iterator((ePid, newEdgePartition)) - } - new GraphImpl(vertices, new EdgeRDD(newEdgePartitions), routingTable, replicatedVertexView) + vertices.cache() + val mapUsesSrcAttr = accessesVertexAttr(f, "srcAttr") + val mapUsesDstAttr = accessesVertexAttr(f, "dstAttr") + replicatedVertexView.upgrade(vertices, mapUsesSrcAttr, mapUsesDstAttr) + val newEdges = replicatedVertexView.edges.mapEdgePartitions { (pid, part) => + part.map(f(pid, part.tripletIterator(mapUsesSrcAttr, mapUsesDstAttr))) + } + new GraphImpl(vertices, replicatedVertexView.withEdges(newEdges)) } override def subgraph( epred: EdgeTriplet[VD, ED] => Boolean = x => true, vpred: (VertexId, VD) => Boolean = (a, b) => true): Graph[VD, ED] = { + vertices.cache() // Filter the vertices, reusing the partitioner and the index from this graph val newVerts = vertices.mapVertexPartitions(_.filter(vpred)) - - // Filter the edges - val edTag = classTag[ED] - val newEdges = new EdgeRDD[ED](triplets.filter { et => - vpred(et.srcId, et.srcAttr) && vpred(et.dstId, et.dstAttr) && epred(et) - }.mapPartitionsWithIndex( { (pid, iter) => - val builder = new EdgePartitionBuilder[ED]()(edTag) - iter.foreach { et => builder.add(et.srcId, et.dstId, et.attr) } - val edgePartition = builder.toEdgePartition - Iterator((pid, edgePartition)) - }, preservesPartitioning = true)).cache() - - // Reuse the previous ReplicatedVertexView unmodified. The replicated vertices that have been - // removed will be ignored, since we only refer to replicated vertices when they are adjacent to - // an edge. - new GraphImpl(newVerts, newEdges, new RoutingTable(newEdges, newVerts), replicatedVertexView) - } // end of subgraph + // Filter the triplets. We must always upgrade the triplet view fully because vpred always runs + // on both src and dst vertices + replicatedVertexView.upgrade(vertices, true, true) + val newEdges = replicatedVertexView.edges.filter(epred, vpred) + new GraphImpl(newVerts, replicatedVertexView.withEdges(newEdges)) + } override def mask[VD2: ClassTag, ED2: ClassTag] ( other: Graph[VD2, ED2]): Graph[VD, ED] = { val newVerts = vertices.innerJoin(other.vertices) { (vid, v, w) => v } - val newEdges = edges.innerJoin(other.edges) { (src, dst, v, w) => v } - // Reuse the previous ReplicatedVertexView unmodified. The replicated vertices that have been - // removed will be ignored, since we only refer to replicated vertices when they are adjacent to - // an edge. - new GraphImpl(newVerts, newEdges, routingTable, replicatedVertexView) + val newEdges = replicatedVertexView.edges.innerJoin(other.edges) { (src, dst, v, w) => v } + new GraphImpl(newVerts, replicatedVertexView.withEdges(newEdges)) } override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { - ClosureCleaner.clean(merge) - val newETable = edges.mapEdgePartitions((pid, part) => part.groupEdges(merge)) - new GraphImpl(vertices, newETable, routingTable, replicatedVertexView) + val newEdges = replicatedVertexView.edges.mapEdgePartitions( + (pid, part) => part.groupEdges(merge)) + new GraphImpl(vertices, replicatedVertexView.withEdges(newEdges)) } // /////////////////////////////////////////////////////////////////////////////////////////////// @@ -199,68 +165,58 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( reduceFunc: (A, A) => A, activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None): VertexRDD[A] = { - ClosureCleaner.clean(mapFunc) - ClosureCleaner.clean(reduceFunc) + vertices.cache() // For each vertex, replicate its attribute only to partitions where it is // in the relevant position in an edge. val mapUsesSrcAttr = accessesVertexAttr(mapFunc, "srcAttr") val mapUsesDstAttr = accessesVertexAttr(mapFunc, "dstAttr") - val vs = activeSetOpt match { + replicatedVertexView.upgrade(vertices, mapUsesSrcAttr, mapUsesDstAttr) + val view = activeSetOpt match { case Some((activeSet, _)) => - replicatedVertexView.get(mapUsesSrcAttr, mapUsesDstAttr, activeSet) + replicatedVertexView.withActiveSet(activeSet) case None => - replicatedVertexView.get(mapUsesSrcAttr, mapUsesDstAttr) + replicatedVertexView } val activeDirectionOpt = activeSetOpt.map(_._2) // Map and combine. - val preAgg = edges.partitionsRDD.zipPartitions(vs, true) { (ePartIter, vPartIter) => - val (ePid, edgePartition) = ePartIter.next() - val (vPid, vPart) = vPartIter.next() - assert(!vPartIter.hasNext) - assert(ePid == vPid) - // Choose scan method - val activeFraction = vPart.numActives.getOrElse(0) / edgePartition.indexSize.toFloat - val edgeIter = activeDirectionOpt match { - case Some(EdgeDirection.Both) => - if (activeFraction < 0.8) { - edgePartition.indexIterator(srcVertexId => vPart.isActive(srcVertexId)) - .filter(e => vPart.isActive(e.dstId)) - } else { - edgePartition.iterator.filter(e => vPart.isActive(e.srcId) && vPart.isActive(e.dstId)) - } - case Some(EdgeDirection.Either) => - // TODO: Because we only have a clustered index on the source vertex ID, we can't filter - // the index here. Instead we have to scan all edges and then do the filter. - edgePartition.iterator.filter(e => vPart.isActive(e.srcId) || vPart.isActive(e.dstId)) - case Some(EdgeDirection.Out) => - if (activeFraction < 0.8) { - edgePartition.indexIterator(srcVertexId => vPart.isActive(srcVertexId)) - } else { - edgePartition.iterator.filter(e => vPart.isActive(e.srcId)) - } - case Some(EdgeDirection.In) => - edgePartition.iterator.filter(e => vPart.isActive(e.dstId)) - case _ => // None - edgePartition.iterator - } - - // Scan edges and run the map function - val et = new EdgeTriplet[VD, ED] - val mapOutputs = edgeIter.flatMap { e => - et.set(e) - if (mapUsesSrcAttr) { - et.srcAttr = vPart(e.srcId) - } - if (mapUsesDstAttr) { - et.dstAttr = vPart(e.dstId) + val preAgg = view.edges.partitionsRDD.mapPartitions(_.flatMap { + case (pid, edgePartition) => + // Choose scan method + val activeFraction = edgePartition.numActives.getOrElse(0) / edgePartition.indexSize.toFloat + val edgeIter = activeDirectionOpt match { + case Some(EdgeDirection.Both) => + if (activeFraction < 0.8) { + edgePartition.indexIterator(srcVertexId => edgePartition.isActive(srcVertexId)) + .filter(e => edgePartition.isActive(e.dstId)) + } else { + edgePartition.iterator.filter(e => + edgePartition.isActive(e.srcId) && edgePartition.isActive(e.dstId)) + } + case Some(EdgeDirection.Either) => + // TODO: Because we only have a clustered index on the source vertex ID, we can't filter + // the index here. Instead we have to scan all edges and then do the filter. + edgePartition.iterator.filter(e => + edgePartition.isActive(e.srcId) || edgePartition.isActive(e.dstId)) + case Some(EdgeDirection.Out) => + if (activeFraction < 0.8) { + edgePartition.indexIterator(srcVertexId => edgePartition.isActive(srcVertexId)) + } else { + edgePartition.iterator.filter(e => edgePartition.isActive(e.srcId)) + } + case Some(EdgeDirection.In) => + edgePartition.iterator.filter(e => edgePartition.isActive(e.dstId)) + case _ => // None + edgePartition.iterator } - mapFunc(et) - } - // Note: This doesn't allow users to send messages to arbitrary vertices. - vPart.aggregateUsingIndex(mapOutputs, reduceFunc).iterator - } + + // Scan edges and run the map function + val mapOutputs = edgePartition.upgradeIterator(edgeIter, mapUsesSrcAttr, mapUsesDstAttr) + .flatMap(mapFunc(_)) + // Note: This doesn't allow users to send messages to arbitrary vertices. + edgePartition.vertices.aggregateUsingIndex(mapOutputs, reduceFunc).iterator + }).setName("GraphImpl.mapReduceTriplets - preAgg") // do the final reduction reusing the index map vertices.aggregateUsingIndex(preAgg, reduceFunc) @@ -268,20 +224,19 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def outerJoinVertices[U: ClassTag, VD2: ClassTag] (other: RDD[(VertexId, U)]) - (updateF: (VertexId, VD, Option[U]) => VD2): Graph[VD2, ED] = - { + (updateF: (VertexId, VD, Option[U]) => VD2): Graph[VD2, ED] = { if (classTag[VD] equals classTag[VD2]) { + vertices.cache() // updateF preserves type, so we can use incremental replication - val newVerts = vertices.leftJoin(other)(updateF) + val newVerts = vertices.leftJoin(other)(updateF).cache() val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) - val newReplicatedVertexView = new ReplicatedVertexView[VD2]( - changedVerts, edges, routingTable, - Some(replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2]])) - new GraphImpl(newVerts, edges, routingTable, newReplicatedVertexView) + val newReplicatedVertexView = replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2, ED]] + .updateVertices(changedVerts) + new GraphImpl(newVerts, newReplicatedVertexView) } else { // updateF does not preserve type, so we must re-replicate all vertices val newVerts = vertices.leftJoin(other)(updateF) - GraphImpl(newVerts, edges, routingTable) + GraphImpl(newVerts, replicatedVertexView.edges) } } @@ -298,73 +253,68 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( object GraphImpl { + /** Create a graph from edges, setting referenced vertices to `defaultVertexAttr`. */ def apply[VD: ClassTag, ED: ClassTag]( edges: RDD[Edge[ED]], - defaultVertexAttr: VD): GraphImpl[VD, ED] = - { + defaultVertexAttr: VD): GraphImpl[VD, ED] = { fromEdgeRDD(createEdgeRDD(edges), defaultVertexAttr) } + /** Create a graph from EdgePartitions, setting referenced vertices to `defaultVertexAttr`. */ def fromEdgePartitions[VD: ClassTag, ED: ClassTag]( - edgePartitions: RDD[(PartitionID, EdgePartition[ED])], + edgePartitions: RDD[(PartitionID, EdgePartition[ED, VD])], defaultVertexAttr: VD): GraphImpl[VD, ED] = { fromEdgeRDD(new EdgeRDD(edgePartitions), defaultVertexAttr) } + /** Create a graph from vertices and edges, setting missing vertices to `defaultVertexAttr`. */ def apply[VD: ClassTag, ED: ClassTag]( vertices: RDD[(VertexId, VD)], edges: RDD[Edge[ED]], - defaultVertexAttr: VD): GraphImpl[VD, ED] = - { - val edgeRDD = createEdgeRDD(edges).cache() - - // Get the set of all vids - val partitioner = Partitioner.defaultPartitioner(vertices) - val vPartitioned = vertices.partitionBy(partitioner) - val vidsFromEdges = collectVertexIdsFromEdges(edgeRDD, partitioner) - val vids = vPartitioned.zipPartitions(vidsFromEdges) { (vertexIter, vidsFromEdgesIter) => - vertexIter.map(_._1) ++ vidsFromEdgesIter.map(_._1) - } - - val vertexRDD = VertexRDD(vids, vPartitioned, defaultVertexAttr) - + defaultVertexAttr: VD): GraphImpl[VD, ED] = { + val edgeRDD = createEdgeRDD(edges)(classTag[ED], classTag[VD]).cache() + val vertexRDD = VertexRDD(vertices, edgeRDD, defaultVertexAttr) GraphImpl(vertexRDD, edgeRDD) } + /** Create a graph from a VertexRDD and an EdgeRDD with arbitrary replicated vertices. */ def apply[VD: ClassTag, ED: ClassTag]( vertices: VertexRDD[VD], - edges: EdgeRDD[ED]): GraphImpl[VD, ED] = { - // Cache RDDs that are referenced multiple times - edges.cache() - - GraphImpl(vertices, edges, new RoutingTable(edges, vertices)) + edges: EdgeRDD[ED, _]): GraphImpl[VD, ED] = { + // Convert the vertex partitions in edges to the correct type + val newEdges = edges.mapEdgePartitions( + (pid, part) => part.withVertices(part.vertices.map( + (vid, attr) => null.asInstanceOf[VD]))) + GraphImpl.fromExistingRDDs(vertices, newEdges) } - def apply[VD: ClassTag, ED: ClassTag]( + /** + * Create a graph from a VertexRDD and an EdgeRDD with the same replicated vertex type as the + * vertices. + */ + def fromExistingRDDs[VD: ClassTag, ED: ClassTag]( vertices: VertexRDD[VD], - edges: EdgeRDD[ED], - routingTable: RoutingTable): GraphImpl[VD, ED] = { - // Cache RDDs that are referenced multiple times. `routingTable` is cached by default, so we - // don't cache it explicitly. - vertices.cache() - edges.cache() - - new GraphImpl( - vertices, edges, routingTable, new ReplicatedVertexView(vertices, edges, routingTable)) + edges: EdgeRDD[ED, VD]): GraphImpl[VD, ED] = { + new GraphImpl(vertices, new ReplicatedVertexView(edges)) } /** - * Create the edge RDD, which is much more efficient for Java heap storage than the normal edges - * data structure (RDD[(VertexId, VertexId, ED)]). - * - * The edge RDD contains multiple partitions, and each partition contains only one RDD key-value - * pair: the key is the partition id, and the value is an EdgePartition object containing all the - * edges in a partition. + * Create a graph from an EdgeRDD with the correct vertex type, setting missing vertices to + * `defaultVertexAttr`. The vertices will have the same number of partitions as the EdgeRDD. */ - private def createEdgeRDD[ED: ClassTag]( - edges: RDD[Edge[ED]]): EdgeRDD[ED] = { + private def fromEdgeRDD[VD: ClassTag, ED: ClassTag]( + edges: EdgeRDD[ED, VD], + defaultVertexAttr: VD): GraphImpl[VD, ED] = { + edges.cache() + val vertices = VertexRDD.fromEdges(edges, edges.partitions.size, defaultVertexAttr) + fromExistingRDDs(vertices, edges) + } + + /** Create an EdgeRDD from a set of edges. */ + private def createEdgeRDD[ED: ClassTag, VD: ClassTag]( + edges: RDD[Edge[ED]]): EdgeRDD[ED, VD] = { val edgePartitions = edges.mapPartitionsWithIndex { (pid, iter) => - val builder = new EdgePartitionBuilder[ED] + val builder = new EdgePartitionBuilder[ED, VD] iter.foreach { e => builder.add(e.srcId, e.dstId, e.attr) } @@ -373,24 +323,4 @@ object GraphImpl { new EdgeRDD(edgePartitions) } - private def fromEdgeRDD[VD: ClassTag, ED: ClassTag]( - edges: EdgeRDD[ED], - defaultVertexAttr: VD): GraphImpl[VD, ED] = { - edges.cache() - // Get the set of all vids - val vids = collectVertexIdsFromEdges(edges, new HashPartitioner(edges.partitions.size)) - // Create the VertexRDD. - val vertices = VertexRDD(vids.mapValues(x => defaultVertexAttr)) - GraphImpl(vertices, edges) - } - - /** Collects all vids mentioned in edges and partitions them by partitioner. */ - private def collectVertexIdsFromEdges( - edges: EdgeRDD[_], - partitioner: Partitioner): RDD[(VertexId, Int)] = { - // TODO: Consider doing map side distinct before shuffle. - new ShuffledRDD[VertexId, Int, (VertexId, Int)]( - edges.collectVertexIds.map(vid => (vid, 0)), partitioner) - .setSerializer(new VertexIdMsgSerializer) - } } // end of object GraphImpl diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala index c45ba3d2f8c24..1c6d7e59e9a27 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -89,7 +89,6 @@ class MsgRDDFunctions[T: ClassTag](self: RDD[MessageToPartition[T]]) { } - private[graphx] object MsgRDDFunctions { implicit def rdd2PartitionRDDFunctions[T: ClassTag](rdd: RDD[MessageToPartition[T]]) = { @@ -99,18 +98,28 @@ object MsgRDDFunctions { implicit def rdd2vertexMessageRDDFunctions[T: ClassTag](rdd: RDD[VertexBroadcastMsg[T]]) = { new VertexBroadcastMsgRDDFunctions(rdd) } +} - def partitionForAggregation[T: ClassTag](msgs: RDD[(VertexId, T)], partitioner: Partitioner) = { - val rdd = new ShuffledRDD[VertexId, T, (VertexId, T)](msgs, partitioner) +private[graphx] +class VertexRDDFunctions[VD: ClassTag](self: RDD[(VertexId, VD)]) { + def copartitionWithVertices(partitioner: Partitioner): RDD[(VertexId, VD)] = { + val rdd = new ShuffledRDD[VertexId, VD, (VertexId, VD)](self, partitioner) // Set a custom serializer if the data is of int or double type. - if (classTag[T] == ClassTag.Int) { + if (classTag[VD] == ClassTag.Int) { rdd.setSerializer(new IntAggMsgSerializer) - } else if (classTag[T] == ClassTag.Long) { + } else if (classTag[VD] == ClassTag.Long) { rdd.setSerializer(new LongAggMsgSerializer) - } else if (classTag[T] == ClassTag.Double) { + } else if (classTag[VD] == ClassTag.Double) { rdd.setSerializer(new DoubleAggMsgSerializer) } rdd } } + +private[graphx] +object VertexRDDFunctions { + implicit def rdd2VertexRDDFunctions[VD: ClassTag](rdd: RDD[(VertexId, VD)]) = { + new VertexRDDFunctions(rdd) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala index a8154b63ce5fb..3a0bba1b93b41 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala @@ -21,192 +21,102 @@ import scala.reflect.{classTag, ClassTag} import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD -import org.apache.spark.util.collection.{PrimitiveVector, OpenHashSet} import org.apache.spark.graphx._ /** - * A view of the vertices after they are shipped to the join sites specified in - * `vertexPlacement`. The resulting view is co-partitioned with `edges`. If `prevViewOpt` is - * specified, `updatedVerts` are treated as incremental updates to the previous view. Otherwise, a - * fresh view is created. - * - * The view is always cached (i.e., once it is evaluated, it remains materialized). This avoids - * constructing it twice if the user calls graph.triplets followed by graph.mapReduceTriplets, for - * example. However, it means iterative algorithms must manually call `Graph.unpersist` on previous - * iterations' graphs for best GC performance. See the implementation of - * [[org.apache.spark.graphx.Pregel]] for an example. + * Manages shipping vertex attributes to the edge partitions of an + * [[org.apache.spark.graphx.EdgeRDD]]. Vertex attributes may be partially shipped to construct a + * triplet view with vertex attributes on only one side, and they may be updated. An active vertex + * set may additionally be shipped to the edge partitions. Be careful not to store a reference to + * `edges`, since it may be modified when the attribute shipping level is upgraded. */ private[impl] -class ReplicatedVertexView[VD: ClassTag]( - updatedVerts: VertexRDD[VD], - edges: EdgeRDD[_], - routingTable: RoutingTable, - prevViewOpt: Option[ReplicatedVertexView[VD]] = None) { +class ReplicatedVertexView[VD: ClassTag, ED: ClassTag]( + var edges: EdgeRDD[ED, VD], + var hasSrcId: Boolean = false, + var hasDstId: Boolean = false) { /** - * Within each edge partition, create a local map from vid to an index into the attribute - * array. Each map contains a superset of the vertices that it will receive, because it stores - * vids from both the source and destination of edges. It must always include both source and - * destination vids because some operations, such as GraphImpl.mapReduceTriplets, rely on this. + * Return a new `ReplicatedVertexView` with the specified `EdgeRDD`, which must have the same + * shipping level. */ - private val localVertexIdMap: RDD[(Int, VertexIdToIndexMap)] = prevViewOpt match { - case Some(prevView) => - prevView.localVertexIdMap - case None => - edges.partitionsRDD.mapPartitions(_.map { - case (pid, epart) => - val vidToIndex = new VertexIdToIndexMap - epart.foreach { e => - vidToIndex.add(e.srcId) - vidToIndex.add(e.dstId) - } - (pid, vidToIndex) - }, preservesPartitioning = true).cache().setName("ReplicatedVertexView localVertexIdMap") - } - - private lazy val bothAttrs: RDD[(PartitionID, VertexPartition[VD])] = create(true, true) - private lazy val srcAttrOnly: RDD[(PartitionID, VertexPartition[VD])] = create(true, false) - private lazy val dstAttrOnly: RDD[(PartitionID, VertexPartition[VD])] = create(false, true) - private lazy val noAttrs: RDD[(PartitionID, VertexPartition[VD])] = create(false, false) - - def unpersist(blocking: Boolean = true): ReplicatedVertexView[VD] = { - bothAttrs.unpersist(blocking) - srcAttrOnly.unpersist(blocking) - dstAttrOnly.unpersist(blocking) - noAttrs.unpersist(blocking) - // Don't unpersist localVertexIdMap because a future ReplicatedVertexView may be using it - // without modification - this + def withEdges[VD2: ClassTag, ED2: ClassTag]( + edges_ : EdgeRDD[ED2, VD2]): ReplicatedVertexView[VD2, ED2] = { + new ReplicatedVertexView(edges_, hasSrcId, hasDstId) } - def get(includeSrc: Boolean, includeDst: Boolean): RDD[(PartitionID, VertexPartition[VD])] = { - (includeSrc, includeDst) match { - case (true, true) => bothAttrs - case (true, false) => srcAttrOnly - case (false, true) => dstAttrOnly - case (false, false) => noAttrs - } + /** + * Return a new `ReplicatedVertexView` where edges are reversed and shipping levels are swapped to + * match. + */ + def reverse() = { + val newEdges = edges.mapEdgePartitions((pid, part) => part.reverse) + new ReplicatedVertexView(newEdges, hasDstId, hasSrcId) } - def get( - includeSrc: Boolean, - includeDst: Boolean, - actives: VertexRDD[_]): RDD[(PartitionID, VertexPartition[VD])] = { - // Ship active sets to edge partitions using vertexPlacement, but ignoring includeSrc and - // includeDst. These flags govern attribute shipping, but the activeness of a vertex must be - // shipped to all edges mentioning that vertex, regardless of whether the vertex attribute is - // also shipped there. - val shippedActives = routingTable.get(true, true) - .zipPartitions(actives.partitionsRDD)(ReplicatedVertexView.buildActiveBuffer(_, _)) - .partitionBy(edges.partitioner.get) - // Update the view with shippedActives, setting activeness flags in the resulting - // VertexPartitions - get(includeSrc, includeDst).zipPartitions(shippedActives) { (viewIter, shippedActivesIter) => - val (pid, vPart) = viewIter.next() - val newPart = vPart.replaceActives(shippedActivesIter.flatMap(_._2.iterator)) - Iterator((pid, newPart)) + /** + * Upgrade the shipping level in-place to the specified levels by shipping vertex attributes from + * `vertices`. This operation modifies the `ReplicatedVertexView`, and callers can access `edges` + * afterwards to obtain the upgraded view. + */ + def upgrade(vertices: VertexRDD[VD], includeSrc: Boolean, includeDst: Boolean) { + val shipSrc = includeSrc && !hasSrcId + val shipDst = includeDst && !hasDstId + if (shipSrc || shipDst) { + val shippedVerts: RDD[(Int, VertexAttributeBlock[VD])] = + vertices.shipVertexAttributes(shipSrc, shipDst) + .setName("ReplicatedVertexView.upgrade(%s, %s) - shippedVerts %s %s (broadcast)".format( + includeSrc, includeDst, shipSrc, shipDst)) + .partitionBy(edges.partitioner.get) + val newEdges = new EdgeRDD(edges.partitionsRDD.zipPartitions(shippedVerts) { + (ePartIter, shippedVertsIter) => ePartIter.map { + case (pid, edgePartition) => + (pid, edgePartition.updateVertices(shippedVertsIter.flatMap(_._2.iterator))) + } + }) + edges = newEdges + hasSrcId = includeSrc + hasDstId = includeDst } } - private def create(includeSrc: Boolean, includeDst: Boolean) - : RDD[(PartitionID, VertexPartition[VD])] = { - val vdTag = classTag[VD] - - // Ship vertex attributes to edge partitions according to vertexPlacement - val verts = updatedVerts.partitionsRDD - val shippedVerts = routingTable.get(includeSrc, includeDst) - .zipPartitions(verts)(ReplicatedVertexView.buildBuffer(_, _)(vdTag)) + /** + * Return a new `ReplicatedVertexView` where the `activeSet` in each edge partition contains only + * vertex ids present in `actives`. This ships a vertex id to all edge partitions where it is + * referenced, ignoring the attribute shipping level. + */ + def withActiveSet(actives: VertexRDD[_]): ReplicatedVertexView[VD, ED] = { + val shippedActives = actives.shipVertexIds() + .setName("ReplicatedVertexView.withActiveSet - shippedActives (broadcast)") .partitionBy(edges.partitioner.get) - // TODO: Consider using a specialized shuffler. - - prevViewOpt match { - case Some(prevView) => - // Update prevView with shippedVerts, setting staleness flags in the resulting - // VertexPartitions - prevView.get(includeSrc, includeDst).zipPartitions(shippedVerts) { - (prevViewIter, shippedVertsIter) => - val (pid, prevVPart) = prevViewIter.next() - val newVPart = prevVPart.innerJoinKeepLeft(shippedVertsIter.flatMap(_._2.iterator)) - Iterator((pid, newVPart)) - }.cache().setName("ReplicatedVertexView delta %s %s".format(includeSrc, includeDst)) - case None => - // Within each edge partition, place the shipped vertex attributes into the correct - // locations specified in localVertexIdMap - localVertexIdMap.zipPartitions(shippedVerts) { (mapIter, shippedVertsIter) => - val (pid, vidToIndex) = mapIter.next() - assert(!mapIter.hasNext) - // Populate the vertex array using the vidToIndex map - val vertexArray = vdTag.newArray(vidToIndex.capacity) - for ((_, block) <- shippedVertsIter) { - for (i <- 0 until block.vids.size) { - val vid = block.vids(i) - val attr = block.attrs(i) - val ind = vidToIndex.getPos(vid) - vertexArray(ind) = attr - } - } - val newVPart = new VertexPartition( - vidToIndex, vertexArray, vidToIndex.getBitSet)(vdTag) - Iterator((pid, newVPart)) - }.cache().setName("ReplicatedVertexView %s %s".format(includeSrc, includeDst)) - } - } -} - -private object ReplicatedVertexView { - protected def buildBuffer[VD: ClassTag]( - pid2vidIter: Iterator[Array[Array[VertexId]]], - vertexPartIter: Iterator[VertexPartition[VD]]) = { - val pid2vid: Array[Array[VertexId]] = pid2vidIter.next() - val vertexPart: VertexPartition[VD] = vertexPartIter.next() - - Iterator.tabulate(pid2vid.size) { pid => - val vidsCandidate = pid2vid(pid) - val size = vidsCandidate.length - val vids = new PrimitiveVector[VertexId](pid2vid(pid).size) - val attrs = new PrimitiveVector[VD](pid2vid(pid).size) - var i = 0 - while (i < size) { - val vid = vidsCandidate(i) - if (vertexPart.isDefined(vid)) { - vids += vid - attrs += vertexPart(vid) - } - i += 1 + val newEdges = new EdgeRDD(edges.partitionsRDD.zipPartitions(shippedActives) { + (ePartIter, shippedActivesIter) => ePartIter.map { + case (pid, edgePartition) => + (pid, edgePartition.withActiveSet(shippedActivesIter.flatMap(_._2.iterator))) } - (pid, new VertexAttributeBlock(vids.trim().array, attrs.trim().array)) - } + }) + new ReplicatedVertexView(newEdges, hasSrcId, hasDstId) } - protected def buildActiveBuffer( - pid2vidIter: Iterator[Array[Array[VertexId]]], - activePartIter: Iterator[VertexPartition[_]]) - : Iterator[(Int, Array[VertexId])] = { - val pid2vid: Array[Array[VertexId]] = pid2vidIter.next() - val activePart: VertexPartition[_] = activePartIter.next() + /** + * Return a new `ReplicatedVertexView` where vertex attributes in edge partition are updated using + * `updates`. This ships a vertex attribute only to the edge partitions where it is in the + * position(s) specified by the attribute shipping level. + */ + def updateVertices(updates: VertexRDD[VD]): ReplicatedVertexView[VD, ED] = { + val shippedVerts = updates.shipVertexAttributes(hasSrcId, hasDstId) + .setName("ReplicatedVertexView.updateVertices - shippedVerts %s %s (broadcast)".format( + hasSrcId, hasDstId)) + .partitionBy(edges.partitioner.get) - Iterator.tabulate(pid2vid.size) { pid => - val vidsCandidate = pid2vid(pid) - val size = vidsCandidate.length - val actives = new PrimitiveVector[VertexId](vidsCandidate.size) - var i = 0 - while (i < size) { - val vid = vidsCandidate(i) - if (activePart.isDefined(vid)) { - actives += vid - } - i += 1 + val newEdges = new EdgeRDD(edges.partitionsRDD.zipPartitions(shippedVerts) { + (ePartIter, shippedVertsIter) => ePartIter.map { + case (pid, edgePartition) => + (pid, edgePartition.updateVertices(shippedVertsIter.flatMap(_._2.iterator))) } - (pid, actives.trim().array) - } + }) + new ReplicatedVertexView(newEdges, hasSrcId, hasDstId) } } - -private[graphx] -class VertexAttributeBlock[VD: ClassTag](val vids: Array[VertexId], val attrs: Array[VD]) - extends Serializable { - def iterator: Iterator[(VertexId, VD)] = - (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) } -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala deleted file mode 100644 index 022d5668e2942..0000000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.graphx.impl - -import org.apache.spark.SparkContext._ -import org.apache.spark.graphx._ -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.collection.PrimitiveVector - -/** - * Stores the locations of edge-partition join sites for each vertex attribute; that is, the routing - * information for shipping vertex attributes to edge partitions. This is always cached because it - * may be used multiple times in ReplicatedVertexView -- once to ship the vertex attributes and - * (possibly) once to ship the active-set information. - */ -private[impl] -class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) { - - val bothAttrs: RDD[Array[Array[VertexId]]] = createPid2Vid(true, true) - val srcAttrOnly: RDD[Array[Array[VertexId]]] = createPid2Vid(true, false) - val dstAttrOnly: RDD[Array[Array[VertexId]]] = createPid2Vid(false, true) - val noAttrs: RDD[Array[Array[VertexId]]] = createPid2Vid(false, false) - - def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[VertexId]]] = - (includeSrcAttr, includeDstAttr) match { - case (true, true) => bothAttrs - case (true, false) => srcAttrOnly - case (false, true) => dstAttrOnly - case (false, false) => noAttrs - } - - private def createPid2Vid( - includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[VertexId]]] = { - // Determine which vertices each edge partition needs by creating a mapping from vid to pid. - val vid2pid: RDD[(VertexId, PartitionID)] = edges.partitionsRDD.mapPartitions { iter => - val (pid: PartitionID, edgePartition: EdgePartition[_]) = iter.next() - val numEdges = edgePartition.size - val vSet = new VertexSet - if (includeSrcAttr) { // Add src vertices to the set. - var i = 0 - while (i < numEdges) { - vSet.add(edgePartition.srcIds(i)) - i += 1 - } - } - if (includeDstAttr) { // Add dst vertices to the set. - var i = 0 - while (i < numEdges) { - vSet.add(edgePartition.dstIds(i)) - i += 1 - } - } - vSet.iterator.map { vid => (vid, pid) } - } - - val numEdgePartitions = edges.partitions.size - vid2pid.partitionBy(vertices.partitioner.get).mapPartitions { iter => - val pid2vid = Array.fill(numEdgePartitions)(new PrimitiveVector[VertexId]) - for ((vid, pid) <- iter) { - pid2vid(pid) += vid - } - - Iterator(pid2vid.map(_.trim().array)) - }.cache().setName("RoutingTable %s %s".format(includeSrcAttr, includeDstAttr)) - } -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala new file mode 100644 index 0000000000000..927e32ad0f448 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx.impl + +import scala.reflect.ClassTag + +import org.apache.spark.Partitioner +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.ShuffledRDD +import org.apache.spark.util.collection.{BitSet, PrimitiveVector} + +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap + +/** + * A message from the edge partition `pid` to the vertex partition containing `vid` specifying that + * the edge partition references `vid` in the specified `position` (src, dst, or both). +*/ +private[graphx] +class RoutingTableMessage( + var vid: VertexId, + var pid: PartitionID, + var position: Byte) + extends Product2[VertexId, (PartitionID, Byte)] with Serializable { + override def _1 = vid + override def _2 = (pid, position) + override def canEqual(that: Any): Boolean = that.isInstanceOf[RoutingTableMessage] +} + +private[graphx] +class RoutingTableMessageRDDFunctions(self: RDD[RoutingTableMessage]) { + /** Copartition an `RDD[RoutingTableMessage]` with the vertex RDD with the given `partitioner`. */ + def copartitionWithVertices(partitioner: Partitioner): RDD[RoutingTableMessage] = { + new ShuffledRDD[VertexId, (PartitionID, Byte), RoutingTableMessage](self, partitioner) + .setSerializer(new RoutingTableMessageSerializer) + } +} + +private[graphx] +object RoutingTableMessageRDDFunctions { + import scala.language.implicitConversions + + implicit def rdd2RoutingTableMessageRDDFunctions(rdd: RDD[RoutingTableMessage]) = { + new RoutingTableMessageRDDFunctions(rdd) + } +} + +private[graphx] +object RoutingTablePartition { + val empty: RoutingTablePartition = new RoutingTablePartition(Array.empty) + + /** Generate a `RoutingTableMessage` for each vertex referenced in `edgePartition`. */ + def edgePartitionToMsgs(pid: PartitionID, edgePartition: EdgePartition[_, _]) + : Iterator[RoutingTableMessage] = { + // Determine which positions each vertex id appears in using a map where the low 2 bits + // represent src and dst + val map = new PrimitiveKeyOpenHashMap[VertexId, Byte] + edgePartition.srcIds.iterator.foreach { srcId => + map.changeValue(srcId, 0x1, (b: Byte) => (b | 0x1).toByte) + } + edgePartition.dstIds.iterator.foreach { dstId => + map.changeValue(dstId, 0x2, (b: Byte) => (b | 0x2).toByte) + } + map.iterator.map { vidAndPosition => + new RoutingTableMessage(vidAndPosition._1, pid, vidAndPosition._2) + } + } + + /** Build a `RoutingTablePartition` from `RoutingTableMessage`s. */ + def fromMsgs(numEdgePartitions: Int, iter: Iterator[RoutingTableMessage]) + : RoutingTablePartition = { + val pid2vid = Array.fill(numEdgePartitions)(new PrimitiveVector[VertexId]) + val srcFlags = Array.fill(numEdgePartitions)(new PrimitiveVector[Boolean]) + val dstFlags = Array.fill(numEdgePartitions)(new PrimitiveVector[Boolean]) + for (msg <- iter) { + pid2vid(msg.pid) += msg.vid + srcFlags(msg.pid) += (msg.position & 0x1) != 0 + dstFlags(msg.pid) += (msg.position & 0x2) != 0 + } + + new RoutingTablePartition(pid2vid.zipWithIndex.map { + case (vids, pid) => (vids.trim().array, toBitSet(srcFlags(pid)), toBitSet(dstFlags(pid))) + }) + } + + /** Compact the given vector of Booleans into a BitSet. */ + private def toBitSet(flags: PrimitiveVector[Boolean]): BitSet = { + val bitset = new BitSet(flags.size) + var i = 0 + while (i < flags.size) { + if (flags(i)) { + bitset.set(i) + } + i += 1 + } + bitset + } +} + +/** + * Stores the locations of edge-partition join sites for each vertex attribute in a particular + * vertex partition. This provides routing information for shipping vertex attributes to edge + * partitions. + */ +private[graphx] +class RoutingTablePartition( + private val routingTable: Array[(Array[VertexId], BitSet, BitSet)]) { + /** The maximum number of edge partitions this `RoutingTablePartition` is built to join with. */ + val numEdgePartitions: Int = routingTable.size + + /** Returns the number of vertices that will be sent to the specified edge partition. */ + def partitionSize(pid: PartitionID): Int = routingTable(pid)._1.size + + /** Returns an iterator over all vertex ids stored in this `RoutingTablePartition`. */ + def iterator: Iterator[VertexId] = routingTable.iterator.flatMap(_._1.iterator) + + /** Returns a new RoutingTablePartition reflecting a reversal of all edge directions. */ + def reverse: RoutingTablePartition = { + new RoutingTablePartition(routingTable.map { + case (vids, srcVids, dstVids) => (vids, dstVids, srcVids) + }) + } + + /** + * Runs `f` on each vertex id to be sent to the specified edge partition. Vertex ids can be + * filtered by the position they have in the edge partition. + */ + def foreachWithinEdgePartition + (pid: PartitionID, includeSrc: Boolean, includeDst: Boolean) + (f: VertexId => Unit) { + val (vidsCandidate, srcVids, dstVids) = routingTable(pid) + val size = vidsCandidate.length + if (includeSrc && includeDst) { + // Avoid checks for performance + vidsCandidate.iterator.foreach(f) + } else if (!includeSrc && !includeDst) { + // Do nothing + } else { + val relevantVids = if (includeSrc) srcVids else dstVids + relevantVids.iterator.foreach { i => f(vidsCandidate(i)) } + } + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala index 1de42eeca1f00..033237f597216 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -27,6 +27,35 @@ import scala.reflect.ClassTag import org.apache.spark.graphx._ import org.apache.spark.serializer._ +private[graphx] +class RoutingTableMessageSerializer extends Serializer with Serializable { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream): SerializationStream = + new ShuffleSerializationStream(s) { + def writeObject[T: ClassTag](t: T): SerializationStream = { + val msg = t.asInstanceOf[RoutingTableMessage] + writeVarLong(msg.vid, optimizePositive = false) + writeUnsignedVarInt(msg.pid) + // TODO: Write only the bottom two bits of msg.position + s.write(msg.position) + this + } + } + + override def deserializeStream(s: InputStream): DeserializationStream = + new ShuffleDeserializationStream(s) { + override def readObject[T: ClassTag](): T = { + val a = readVarLong(optimizePositive = false) + val b = readUnsignedVarInt() + val c = s.read() + if (c == -1) throw new EOFException + new RoutingTableMessage(a, b, c.toByte).asInstanceOf[T] + } + } + } +} + private[graphx] class VertexIdMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala new file mode 100644 index 0000000000000..f4e221d4e05ae --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx.impl + +import scala.reflect.ClassTag + +import org.apache.spark.util.collection.{BitSet, PrimitiveVector} + +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap + +/** Stores vertex attributes to ship to an edge partition. */ +private[graphx] +class VertexAttributeBlock[VD: ClassTag](val vids: Array[VertexId], val attrs: Array[VD]) + extends Serializable { + def iterator: Iterator[(VertexId, VD)] = + (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) } +} + +private[graphx] +object ShippableVertexPartition { + /** Construct a `ShippableVertexPartition` from the given vertices without any routing table. */ + def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)]): ShippableVertexPartition[VD] = + apply(iter, RoutingTablePartition.empty, null.asInstanceOf[VD]) + + /** + * Construct a `ShippableVertexPartition` from the given vertices with the specified routing + * table, filling in missing vertices mentioned in the routing table using `defaultVal`. + */ + def apply[VD: ClassTag]( + iter: Iterator[(VertexId, VD)], routingTable: RoutingTablePartition, defaultVal: VD) + : ShippableVertexPartition[VD] = { + val fullIter = iter ++ routingTable.iterator.map(vid => (vid, defaultVal)) + val (index, values, mask) = VertexPartitionBase.initFrom(fullIter, (a: VD, b: VD) => a) + new ShippableVertexPartition(index, values, mask, routingTable) + } + + import scala.language.implicitConversions + + /** + * Implicit conversion to allow invoking `VertexPartitionBase` operations directly on a + * `ShippableVertexPartition`. + */ + implicit def shippablePartitionToOps[VD: ClassTag](partition: ShippableVertexPartition[VD]) = + new ShippableVertexPartitionOps(partition) + + /** + * Implicit evidence that `ShippableVertexPartition` is a member of the + * `VertexPartitionBaseOpsConstructor` typeclass. This enables invoking `VertexPartitionBase` + * operations on a `ShippableVertexPartition` via an evidence parameter, as in + * [[VertexPartitionBaseOps]]. + */ + implicit object ShippableVertexPartitionOpsConstructor + extends VertexPartitionBaseOpsConstructor[ShippableVertexPartition] { + def toOps[VD: ClassTag](partition: ShippableVertexPartition[VD]) + : VertexPartitionBaseOps[VD, ShippableVertexPartition] = shippablePartitionToOps(partition) + } +} + +/** + * A map from vertex id to vertex attribute that additionally stores edge partition join sites for + * each vertex attribute, enabling joining with an [[org.apache.spark.graphx.EdgeRDD]]. + */ +private[graphx] +class ShippableVertexPartition[VD: ClassTag]( + val index: VertexIdToIndexMap, + val values: Array[VD], + val mask: BitSet, + val routingTable: RoutingTablePartition) + extends VertexPartitionBase[VD] { + + /** Return a new ShippableVertexPartition with the specified routing table. */ + def withRoutingTable(routingTable_ : RoutingTablePartition): ShippableVertexPartition[VD] = { + new ShippableVertexPartition(index, values, mask, routingTable_) + } + + /** + * Generate a `VertexAttributeBlock` for each edge partition keyed on the edge partition ID. The + * `VertexAttributeBlock` contains the vertex attributes from the current partition that are + * referenced in the specified positions in the edge partition. + */ + def shipVertexAttributes( + shipSrc: Boolean, shipDst: Boolean): Iterator[(PartitionID, VertexAttributeBlock[VD])] = { + Iterator.tabulate(routingTable.numEdgePartitions) { pid => + val initialSize = if (shipSrc && shipDst) routingTable.partitionSize(pid) else 64 + val vids = new PrimitiveVector[VertexId](initialSize) + val attrs = new PrimitiveVector[VD](initialSize) + var i = 0 + routingTable.foreachWithinEdgePartition(pid, shipSrc, shipDst) { vid => + if (isDefined(vid)) { + vids += vid + attrs += this(vid) + } + i += 1 + } + (pid, new VertexAttributeBlock(vids.trim().array, attrs.trim().array)) + } + } + + /** + * Generate a `VertexId` array for each edge partition keyed on the edge partition ID. The array + * contains the visible vertex ids from the current partition that are referenced in the edge + * partition. + */ + def shipVertexIds(): Iterator[(PartitionID, Array[VertexId])] = { + Iterator.tabulate(routingTable.numEdgePartitions) { pid => + val vids = new PrimitiveVector[VertexId](routingTable.partitionSize(pid)) + var i = 0 + routingTable.foreachWithinEdgePartition(pid, true, true) { vid => + if (isDefined(vid)) { + vids += vid + } + i += 1 + } + (pid, vids.trim().array) + } + } +} + +private[graphx] class ShippableVertexPartitionOps[VD: ClassTag](self: ShippableVertexPartition[VD]) + extends VertexPartitionBaseOps[VD, ShippableVertexPartition](self) { + + def withIndex(index: VertexIdToIndexMap): ShippableVertexPartition[VD] = { + new ShippableVertexPartition(index, self.values, self.mask, self.routingTable) + } + + def withValues[VD2: ClassTag](values: Array[VD2]): ShippableVertexPartition[VD2] = { + new ShippableVertexPartition(self.index, values, self.mask, self.routingTable) + } + + def withMask(mask: BitSet): ShippableVertexPartition[VD] = { + new ShippableVertexPartition(self.index, self.values, mask, self.routingTable) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index 7a54b413dc8ca..f1d174720a1ba 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -19,260 +19,59 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag -import org.apache.spark.Logging +import org.apache.spark.util.collection.BitSet + import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap -import org.apache.spark.util.collection.BitSet private[graphx] object VertexPartition { - - def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)]): VertexPartition[VD] = { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] - iter.foreach { case (k, v) => - map(k) = v - } - new VertexPartition(map.keySet, map._values, map.keySet.getBitSet) - } - - def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD) - : VertexPartition[VD] = - { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] - iter.foreach { case (k, v) => - map.setMerge(k, v, mergeFunc) - } - new VertexPartition(map.keySet, map._values, map.keySet.getBitSet) - } -} - - -private[graphx] -class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( - val index: VertexIdToIndexMap, - val values: Array[VD], - val mask: BitSet, - /** A set of vids of active vertices. May contain vids not in index due to join rewrite. */ - private val activeSet: Option[VertexSet] = None) - extends Logging { - - val capacity: Int = index.capacity - - def size: Int = mask.cardinality() - - /** Return the vertex attribute for the given vertex ID. */ - def apply(vid: VertexId): VD = values(index.getPos(vid)) - - def isDefined(vid: VertexId): Boolean = { - val pos = index.getPos(vid) - pos >= 0 && mask.get(pos) - } - - /** Look up vid in activeSet, throwing an exception if it is None. */ - def isActive(vid: VertexId): Boolean = { - activeSet.get.contains(vid) + /** Construct a `VertexPartition` from the given vertices. */ + def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)]) + : VertexPartition[VD] = { + val (index, values, mask) = VertexPartitionBase.initFrom(iter) + new VertexPartition(index, values, mask) } - /** The number of active vertices, if any exist. */ - def numActives: Option[Int] = activeSet.map(_.size) + import scala.language.implicitConversions /** - * Pass each vertex attribute along with the vertex id through a map - * function and retain the original RDD's partitioning and index. - * - * @tparam VD2 the type returned by the map function - * - * @param f the function applied to each vertex id and vertex - * attribute in the RDD - * - * @return a new VertexPartition with values obtained by applying `f` to - * each of the entries in the original VertexRDD. The resulting - * VertexPartition retains the same index. + * Implicit conversion to allow invoking `VertexPartitionBase` operations directly on a + * `VertexPartition`. */ - def map[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexPartition[VD2] = { - // Construct a view of the map transformation - val newValues = new Array[VD2](capacity) - var i = mask.nextSetBit(0) - while (i >= 0) { - newValues(i) = f(index.getValue(i), values(i)) - i = mask.nextSetBit(i + 1) - } - new VertexPartition[VD2](index, newValues, mask) - } - - /** - * Restrict the vertex set to the set of vertices satisfying the given predicate. - * - * @param pred the user defined predicate - * - * @note The vertex set preserves the original index structure which means that the returned - * RDD can be easily joined with the original vertex-set. Furthermore, the filter only - * modifies the bitmap index and so no new values are allocated. - */ - def filter(pred: (VertexId, VD) => Boolean): VertexPartition[VD] = { - // Allocate the array to store the results into - val newMask = new BitSet(capacity) - // Iterate over the active bits in the old mask and evaluate the predicate - var i = mask.nextSetBit(0) - while (i >= 0) { - if (pred(index.getValue(i), values(i))) { - newMask.set(i) - } - i = mask.nextSetBit(i + 1) - } - new VertexPartition(index, values, newMask) - } + implicit def partitionToOps[VD: ClassTag](partition: VertexPartition[VD]) = + new VertexPartitionOps(partition) /** - * Hides vertices that are the same between this and other. For vertices that are different, keeps - * the values from `other`. The indices of `this` and `other` must be the same. + * Implicit evidence that `VertexPartition` is a member of the `VertexPartitionBaseOpsConstructor` + * typeclass. This enables invoking `VertexPartitionBase` operations on a `VertexPartition` via an + * evidence parameter, as in [[VertexPartitionBaseOps]]. */ - def diff(other: VertexPartition[VD]): VertexPartition[VD] = { - if (index != other.index) { - logWarning("Diffing two VertexPartitions with different indexes is slow.") - diff(createUsingIndex(other.iterator)) - } else { - val newMask = mask & other.mask - var i = newMask.nextSetBit(0) - while (i >= 0) { - if (values(i) == other.values(i)) { - newMask.unset(i) - } - i = newMask.nextSetBit(i + 1) - } - new VertexPartition(index, other.values, newMask) - } - } - - /** Left outer join another VertexPartition. */ - def leftJoin[VD2: ClassTag, VD3: ClassTag] - (other: VertexPartition[VD2]) - (f: (VertexId, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { - if (index != other.index) { - logWarning("Joining two VertexPartitions with different indexes is slow.") - leftJoin(createUsingIndex(other.iterator))(f) - } else { - val newValues = new Array[VD3](capacity) - - var i = mask.nextSetBit(0) - while (i >= 0) { - val otherV: Option[VD2] = if (other.mask.get(i)) Some(other.values(i)) else None - newValues(i) = f(index.getValue(i), values(i), otherV) - i = mask.nextSetBit(i + 1) - } - new VertexPartition(index, newValues, mask) - } - } - - /** Left outer join another iterator of messages. */ - def leftJoin[VD2: ClassTag, VD3: ClassTag] - (other: Iterator[(VertexId, VD2)]) - (f: (VertexId, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { - leftJoin(createUsingIndex(other))(f) - } - - /** Inner join another VertexPartition. */ - def innerJoin[U: ClassTag, VD2: ClassTag](other: VertexPartition[U]) - (f: (VertexId, VD, U) => VD2): VertexPartition[VD2] = { - if (index != other.index) { - logWarning("Joining two VertexPartitions with different indexes is slow.") - innerJoin(createUsingIndex(other.iterator))(f) - } else { - val newMask = mask & other.mask - val newValues = new Array[VD2](capacity) - var i = newMask.nextSetBit(0) - while (i >= 0) { - newValues(i) = f(index.getValue(i), values(i), other.values(i)) - i = newMask.nextSetBit(i + 1) - } - new VertexPartition(index, newValues, newMask) - } - } - - /** - * Inner join an iterator of messages. - */ - def innerJoin[U: ClassTag, VD2: ClassTag] - (iter: Iterator[Product2[VertexId, U]]) - (f: (VertexId, VD, U) => VD2): VertexPartition[VD2] = { - innerJoin(createUsingIndex(iter))(f) + implicit object VertexPartitionOpsConstructor + extends VertexPartitionBaseOpsConstructor[VertexPartition] { + def toOps[VD: ClassTag](partition: VertexPartition[VD]) + : VertexPartitionBaseOps[VD, VertexPartition] = partitionToOps(partition) } +} - /** - * Similar effect as aggregateUsingIndex((a, b) => a) - */ - def createUsingIndex[VD2: ClassTag](iter: Iterator[Product2[VertexId, VD2]]) - : VertexPartition[VD2] = { - val newMask = new BitSet(capacity) - val newValues = new Array[VD2](capacity) - iter.foreach { case (vid, vdata) => - val pos = index.getPos(vid) - if (pos >= 0) { - newMask.set(pos) - newValues(pos) = vdata - } - } - new VertexPartition[VD2](index, newValues, newMask) - } +/** A map from vertex id to vertex attribute. */ +private[graphx] class VertexPartition[VD: ClassTag]( + val index: VertexIdToIndexMap, + val values: Array[VD], + val mask: BitSet) + extends VertexPartitionBase[VD] - /** - * Similar to innerJoin, but vertices from the left side that don't appear in iter will remain in - * the partition, hidden by the bitmask. - */ - def innerJoinKeepLeft(iter: Iterator[Product2[VertexId, VD]]): VertexPartition[VD] = { - val newMask = new BitSet(capacity) - val newValues = new Array[VD](capacity) - System.arraycopy(values, 0, newValues, 0, newValues.length) - iter.foreach { case (vid, vdata) => - val pos = index.getPos(vid) - if (pos >= 0) { - newMask.set(pos) - newValues(pos) = vdata - } - } - new VertexPartition(index, newValues, newMask) - } +private[graphx] class VertexPartitionOps[VD: ClassTag](self: VertexPartition[VD]) + extends VertexPartitionBaseOps[VD, VertexPartition](self) { - def aggregateUsingIndex[VD2: ClassTag]( - iter: Iterator[Product2[VertexId, VD2]], - reduceFunc: (VD2, VD2) => VD2): VertexPartition[VD2] = { - val newMask = new BitSet(capacity) - val newValues = new Array[VD2](capacity) - iter.foreach { product => - val vid = product._1 - val vdata = product._2 - val pos = index.getPos(vid) - if (pos >= 0) { - if (newMask.get(pos)) { - newValues(pos) = reduceFunc(newValues(pos), vdata) - } else { // otherwise just store the new value - newMask.set(pos) - newValues(pos) = vdata - } - } - } - new VertexPartition[VD2](index, newValues, newMask) + def withIndex(index: VertexIdToIndexMap): VertexPartition[VD] = { + new VertexPartition(index, self.values, self.mask) } - def replaceActives(iter: Iterator[VertexId]): VertexPartition[VD] = { - val newActiveSet = new VertexSet - iter.foreach(newActiveSet.add(_)) - new VertexPartition(index, values, mask, Some(newActiveSet)) + def withValues[VD2: ClassTag](values: Array[VD2]): VertexPartition[VD2] = { + new VertexPartition(self.index, values, self.mask) } - /** - * Construct a new VertexPartition whose index contains only the vertices in the mask. - */ - def reindex(): VertexPartition[VD] = { - val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] - val arbitraryMerge = (a: VD, b: VD) => a - for ((k, v) <- this.iterator) { - hashMap.setMerge(k, v, arbitraryMerge) - } - new VertexPartition(hashMap.keySet, hashMap._values, hashMap.keySet.getBitSet) + def withMask(mask: BitSet): VertexPartition[VD] = { + new VertexPartition(self.index, self.values, mask) } - - def iterator: Iterator[(VertexId, VD)] = - mask.iterator.map(ind => (index.getValue(ind), values(ind))) - - def vidIterator: Iterator[VertexId] = mask.iterator.map(ind => index.getValue(ind)) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala new file mode 100644 index 0000000000000..8d9e0204d27f2 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx.impl + +import scala.language.higherKinds +import scala.reflect.ClassTag + +import org.apache.spark.util.collection.BitSet + +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap + +private[graphx] object VertexPartitionBase { + /** + * Construct the constituents of a VertexPartitionBase from the given vertices, merging duplicate + * entries arbitrarily. + */ + def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)]) + : (VertexIdToIndexMap, Array[VD], BitSet) = { + val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + iter.foreach { pair => + map(pair._1) = pair._2 + } + (map.keySet, map._values, map.keySet.getBitSet) + } + + /** + * Construct the constituents of a VertexPartitionBase from the given vertices, merging duplicate + * entries using `mergeFunc`. + */ + def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD) + : (VertexIdToIndexMap, Array[VD], BitSet) = { + val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + iter.foreach { pair => + map.setMerge(pair._1, pair._2, mergeFunc) + } + (map.keySet, map._values, map.keySet.getBitSet) + } +} + +/** + * An abstract map from vertex id to vertex attribute. [[VertexPartition]] is the corresponding + * concrete implementation. [[VertexPartitionBaseOps]] provides a variety of operations for + * VertexPartitionBase and subclasses that provide implicit evidence of membership in the + * `VertexPartitionBaseOpsConstructor` typeclass (for example, + * [[VertexPartition.VertexPartitionOpsConstructor]]). + */ +private[graphx] abstract class VertexPartitionBase[@specialized(Long, Int, Double) VD: ClassTag] { + + def index: VertexIdToIndexMap + def values: Array[VD] + def mask: BitSet + + val capacity: Int = index.capacity + + def size: Int = mask.cardinality() + + /** Return the vertex attribute for the given vertex ID. */ + def apply(vid: VertexId): VD = values(index.getPos(vid)) + + def isDefined(vid: VertexId): Boolean = { + val pos = index.getPos(vid) + pos >= 0 && mask.get(pos) + } + + def iterator: Iterator[(VertexId, VD)] = + mask.iterator.map(ind => (index.getValue(ind), values(ind))) +} + +/** + * A typeclass for subclasses of `VertexPartitionBase` representing the ability to wrap them in a + * `VertexPartitionBaseOps`. + */ +private[graphx] trait VertexPartitionBaseOpsConstructor[T[X] <: VertexPartitionBase[X]] { + def toOps[VD: ClassTag](partition: T[VD]): VertexPartitionBaseOps[VD, T] +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala new file mode 100644 index 0000000000000..21ff615feca6c --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx.impl + +import scala.language.higherKinds +import scala.language.implicitConversions +import scala.reflect.ClassTag + +import org.apache.spark.Logging +import org.apache.spark.util.collection.BitSet + +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap + +/** + * An class containing additional operations for subclasses of VertexPartitionBase that provide + * implicit evidence of membership in the `VertexPartitionBaseOpsConstructor` typeclass (for + * example, [[VertexPartition.VertexPartitionOpsConstructor]]). + */ +private[graphx] abstract class VertexPartitionBaseOps + [VD: ClassTag, Self[X] <: VertexPartitionBase[X] : VertexPartitionBaseOpsConstructor] + (self: Self[VD]) + extends Logging { + + def withIndex(index: VertexIdToIndexMap): Self[VD] + def withValues[VD2: ClassTag](values: Array[VD2]): Self[VD2] + def withMask(mask: BitSet): Self[VD] + + /** + * Pass each vertex attribute along with the vertex id through a map + * function and retain the original RDD's partitioning and index. + * + * @tparam VD2 the type returned by the map function + * + * @param f the function applied to each vertex id and vertex + * attribute in the RDD + * + * @return a new VertexPartition with values obtained by applying `f` to + * each of the entries in the original VertexRDD. The resulting + * VertexPartition retains the same index. + */ + def map[VD2: ClassTag](f: (VertexId, VD) => VD2): Self[VD2] = { + // Construct a view of the map transformation + val newValues = new Array[VD2](self.capacity) + var i = self.mask.nextSetBit(0) + while (i >= 0) { + newValues(i) = f(self.index.getValue(i), self.values(i)) + i = self.mask.nextSetBit(i + 1) + } + this.withValues(newValues) + } + + /** + * Restrict the vertex set to the set of vertices satisfying the given predicate. + * + * @param pred the user defined predicate + * + * @note The vertex set preserves the original index structure which means that the returned + * RDD can be easily joined with the original vertex-set. Furthermore, the filter only + * modifies the bitmap index and so no new values are allocated. + */ + def filter(pred: (VertexId, VD) => Boolean): Self[VD] = { + // Allocate the array to store the results into + val newMask = new BitSet(self.capacity) + // Iterate over the active bits in the old mask and evaluate the predicate + var i = self.mask.nextSetBit(0) + while (i >= 0) { + if (pred(self.index.getValue(i), self.values(i))) { + newMask.set(i) + } + i = self.mask.nextSetBit(i + 1) + } + this.withMask(newMask) + } + + /** + * Hides vertices that are the same between this and other. For vertices that are different, keeps + * the values from `other`. The indices of `this` and `other` must be the same. + */ + def diff(other: Self[VD]): Self[VD] = { + if (self.index != other.index) { + logWarning("Diffing two VertexPartitions with different indexes is slow.") + diff(createUsingIndex(other.iterator)) + } else { + val newMask = self.mask & other.mask + var i = newMask.nextSetBit(0) + while (i >= 0) { + if (self.values(i) == other.values(i)) { + newMask.unset(i) + } + i = newMask.nextSetBit(i + 1) + } + this.withValues(other.values).withMask(newMask) + } + } + + /** Left outer join another VertexPartition. */ + def leftJoin[VD2: ClassTag, VD3: ClassTag] + (other: Self[VD2]) + (f: (VertexId, VD, Option[VD2]) => VD3): Self[VD3] = { + if (self.index != other.index) { + logWarning("Joining two VertexPartitions with different indexes is slow.") + leftJoin(createUsingIndex(other.iterator))(f) + } else { + val newValues = new Array[VD3](self.capacity) + + var i = self.mask.nextSetBit(0) + while (i >= 0) { + val otherV: Option[VD2] = if (other.mask.get(i)) Some(other.values(i)) else None + newValues(i) = f(self.index.getValue(i), self.values(i), otherV) + i = self.mask.nextSetBit(i + 1) + } + this.withValues(newValues) + } + } + + /** Left outer join another iterator of messages. */ + def leftJoin[VD2: ClassTag, VD3: ClassTag] + (other: Iterator[(VertexId, VD2)]) + (f: (VertexId, VD, Option[VD2]) => VD3): Self[VD3] = { + leftJoin(createUsingIndex(other))(f) + } + + /** Inner join another VertexPartition. */ + def innerJoin[U: ClassTag, VD2: ClassTag] + (other: Self[U]) + (f: (VertexId, VD, U) => VD2): Self[VD2] = { + if (self.index != other.index) { + logWarning("Joining two VertexPartitions with different indexes is slow.") + innerJoin(createUsingIndex(other.iterator))(f) + } else { + val newMask = self.mask & other.mask + val newValues = new Array[VD2](self.capacity) + var i = newMask.nextSetBit(0) + while (i >= 0) { + newValues(i) = f(self.index.getValue(i), self.values(i), other.values(i)) + i = newMask.nextSetBit(i + 1) + } + this.withValues(newValues).withMask(newMask) + } + } + + /** + * Inner join an iterator of messages. + */ + def innerJoin[U: ClassTag, VD2: ClassTag] + (iter: Iterator[Product2[VertexId, U]]) + (f: (VertexId, VD, U) => VD2): Self[VD2] = { + innerJoin(createUsingIndex(iter))(f) + } + + /** + * Similar effect as aggregateUsingIndex((a, b) => a) + */ + def createUsingIndex[VD2: ClassTag](iter: Iterator[Product2[VertexId, VD2]]) + : Self[VD2] = { + val newMask = new BitSet(self.capacity) + val newValues = new Array[VD2](self.capacity) + iter.foreach { pair => + val pos = self.index.getPos(pair._1) + if (pos >= 0) { + newMask.set(pos) + newValues(pos) = pair._2 + } + } + this.withValues(newValues).withMask(newMask) + } + + /** + * Similar to innerJoin, but vertices from the left side that don't appear in iter will remain in + * the partition, hidden by the bitmask. + */ + def innerJoinKeepLeft(iter: Iterator[Product2[VertexId, VD]]): Self[VD] = { + val newMask = new BitSet(self.capacity) + val newValues = new Array[VD](self.capacity) + System.arraycopy(self.values, 0, newValues, 0, newValues.length) + iter.foreach { pair => + val pos = self.index.getPos(pair._1) + if (pos >= 0) { + newMask.set(pos) + newValues(pos) = pair._2 + } + } + this.withValues(newValues).withMask(newMask) + } + + def aggregateUsingIndex[VD2: ClassTag]( + iter: Iterator[Product2[VertexId, VD2]], + reduceFunc: (VD2, VD2) => VD2): Self[VD2] = { + val newMask = new BitSet(self.capacity) + val newValues = new Array[VD2](self.capacity) + iter.foreach { product => + val vid = product._1 + val vdata = product._2 + val pos = self.index.getPos(vid) + if (pos >= 0) { + if (newMask.get(pos)) { + newValues(pos) = reduceFunc(newValues(pos), vdata) + } else { // otherwise just store the new value + newMask.set(pos) + newValues(pos) = vdata + } + } + } + this.withValues(newValues).withMask(newMask) + } + + /** + * Construct a new VertexPartition whose index contains only the vertices in the mask. + */ + def reindex(): Self[VD] = { + val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] + val arbitraryMerge = (a: VD, b: VD) => a + for ((k, v) <- self.iterator) { + hashMap.setMerge(k, v, arbitraryMerge) + } + this.withIndex(hashMap.keySet).withValues(hashMap._values).withMask(hashMap.keySet.getBitSet) + } + + /** + * Converts a vertex partition (in particular, one of type `Self`) into a + * `VertexPartitionBaseOps`. Within this class, this allows chaining the methods defined above, + * because these methods return a `Self` and this implicit conversion re-wraps that in a + * `VertexPartitionBaseOps`. This relies on the context bound on `Self`. + */ + private implicit def toOps[VD2: ClassTag]( + partition: Self[VD2]): VertexPartitionBaseOps[VD2, Self] = { + implicitly[VertexPartitionBaseOpsConstructor[Self]].toOps(partition) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala index d901d4fe225fe..069e042ed94a3 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala @@ -55,6 +55,7 @@ object Analytics extends Logging { val conf = new SparkConf() .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + .set("spark.locality.wait", "100000") taskType match { case "pagerank" => @@ -62,12 +63,14 @@ object Analytics extends Logging { var outFname = "" var numEPart = 4 var partitionStrategy: Option[PartitionStrategy] = None + var numIterOpt: Option[Int] = None options.foreach{ case ("tol", v) => tol = v.toFloat case ("output", v) => outFname = v case ("numEPart", v) => numEPart = v.toInt case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) + case ("numIter", v) => numIterOpt = Some(v.toInt) case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } @@ -84,7 +87,10 @@ object Analytics extends Logging { println("GRAPHX: Number of vertices " + graph.vertices.count) println("GRAPHX: Number of edges " + graph.edges.count) - val pr = graph.pageRank(tol).vertices.cache() + val pr = (numIterOpt match { + case Some(numIter) => PageRank.run(graph, numIter) + case None => PageRank.runUntilConvergence(graph, tol) + }).vertices.cache() println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_ + _)) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index 32b5fe4813594..7b9bac5d9c8ea 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -110,7 +110,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val p = 100 val verts = 1 to n val graph = Graph.fromEdgeTuples(sc.parallelize(verts.flatMap(x => - verts.filter(y => y % x == 0).map(y => (x: VertexId, y: VertexId))), p), 0) + verts.withFilter(y => y % x == 0).map(y => (x: VertexId, y: VertexId))), p), 0) assert(graph.edges.partitions.length === p) val partitionedGraph = graph.partitionBy(EdgePartition2D) assert(graph.edges.partitions.length === p) @@ -120,7 +120,13 @@ class GraphSuite extends FunSuite with LocalSparkContext { val part = iter.next()._2 Iterator((part.srcIds ++ part.dstIds).toSet) }.collect - assert(verts.forall(id => partitionSets.count(_.contains(id)) <= bound)) + if (!verts.forall(id => partitionSets.count(_.contains(id)) <= bound)) { + val numFailures = verts.count(id => partitionSets.count(_.contains(id)) > bound) + val failure = verts.maxBy(id => partitionSets.count(_.contains(id))) + fail(("Replication bound test failed for %d/%d vertices. " + + "Example: vertex %d replicated to %d (> %f) partitions.").format( + numFailures, n, failure, partitionSets.count(_.contains(failure)), bound)) + } // This should not be true for the default hash partitioning val partitionSetsUnpartitioned = graph.edges.partitionsRDD.mapPartitions { iter => val part = iter.next()._2 diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index e135d1d7ad6a3..d2e0c01bc35ef 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -26,10 +26,16 @@ import org.apache.spark.graphx._ class EdgePartitionSuite extends FunSuite { + def makeEdgePartition[A: ClassTag](xs: Iterable[(Int, Int, A)]): EdgePartition[A, Int] = { + val builder = new EdgePartitionBuilder[A, Int] + for ((src, dst, attr) <- xs) { builder.add(src: VertexId, dst: VertexId, attr) } + builder.toEdgePartition + } + test("reverse") { val edges = List(Edge(0, 1, 0), Edge(1, 2, 0), Edge(2, 0, 0)) val reversedEdges = List(Edge(0, 2, 0), Edge(1, 0, 0), Edge(2, 1, 0)) - val builder = new EdgePartitionBuilder[Int] + val builder = new EdgePartitionBuilder[Int, Nothing] for (e <- edges) { builder.add(e.srcId, e.dstId, e.attr) } @@ -40,7 +46,7 @@ class EdgePartitionSuite extends FunSuite { test("map") { val edges = List(Edge(0, 1, 0), Edge(1, 2, 0), Edge(2, 0, 0)) - val builder = new EdgePartitionBuilder[Int] + val builder = new EdgePartitionBuilder[Int, Nothing] for (e <- edges) { builder.add(e.srcId, e.dstId, e.attr) } @@ -49,11 +55,22 @@ class EdgePartitionSuite extends FunSuite { edges.map(e => e.copy(attr = e.srcId + e.dstId))) } + test("filter") { + val edges = List(Edge(0, 1, 0), Edge(0, 2, 0), Edge(2, 0, 0)) + val builder = new EdgePartitionBuilder[Int, Int] + for (e <- edges) { + builder.add(e.srcId, e.dstId, e.attr) + } + val edgePartition = builder.toEdgePartition + val filtered = edgePartition.filter(et => et.srcId == 0, (vid, attr) => vid == 0 || vid == 1) + assert(filtered.tripletIterator().toList.map(et => (et.srcId, et.dstId)) === List((0L, 1L))) + } + test("groupEdges") { val edges = List( Edge(0, 1, 1), Edge(1, 2, 2), Edge(2, 0, 4), Edge(0, 1, 8), Edge(1, 2, 16), Edge(2, 0, 32)) val groupedEdges = List(Edge(0, 1, 9), Edge(1, 2, 18), Edge(2, 0, 36)) - val builder = new EdgePartitionBuilder[Int] + val builder = new EdgePartitionBuilder[Int, Nothing] for (e <- edges) { builder.add(e.srcId, e.dstId, e.attr) } @@ -61,11 +78,19 @@ class EdgePartitionSuite extends FunSuite { assert(edgePartition.groupEdges(_ + _).iterator.map(_.copy()).toList === groupedEdges) } + test("upgradeIterator") { + val edges = List((0, 1, 0), (1, 0, 0)) + val verts = List((0L, 1), (1L, 2)) + val part = makeEdgePartition(edges).updateVertices(verts.iterator) + assert(part.upgradeIterator(part.iterator).map(_.toTuple).toList === + part.tripletIterator().toList.map(_.toTuple)) + } + test("indexIterator") { val edgesFrom0 = List(Edge(0, 1, 0)) val edgesFrom1 = List(Edge(1, 0, 0), Edge(1, 2, 0)) val sortedEdges = edgesFrom0 ++ edgesFrom1 - val builder = new EdgePartitionBuilder[Int] + val builder = new EdgePartitionBuilder[Int, Nothing] for (e <- Random.shuffle(sortedEdges)) { builder.add(e.srcId, e.dstId, e.attr) } @@ -77,11 +102,6 @@ class EdgePartitionSuite extends FunSuite { } test("innerJoin") { - def makeEdgePartition[A: ClassTag](xs: Iterable[(Int, Int, A)]): EdgePartition[A] = { - val builder = new EdgePartitionBuilder[A] - for ((src, dst, attr) <- xs) { builder.add(src: VertexId, dst: VertexId, attr) } - builder.toEdgePartition - } val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) val bList = List((0, 1, 0), (1, 0, 0), (1, 1, 0), (3, 4, 0), (5, 5, 0)) val a = makeEdgePartition(aList) @@ -90,4 +110,14 @@ class EdgePartitionSuite extends FunSuite { assert(a.innerJoin(b) { (src, dst, a, b) => a }.iterator.map(_.copy()).toList === List(Edge(0, 1, 0), Edge(1, 0, 0), Edge(5, 5, 0))) } + + test("isActive, numActives, replaceActives") { + val ep = new EdgePartitionBuilder[Nothing, Nothing].toEdgePartition + .withActiveSet(Iterator(0L, 2L, 0L)) + assert(ep.isActive(0)) + assert(!ep.isActive(1)) + assert(ep.isActive(2)) + assert(!ep.isActive(-1)) + assert(ep.numActives == Some(2)) + } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala index 9cbb2d2acdc2d..49b2704390fea 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala @@ -26,17 +26,11 @@ import org.apache.spark.graphx._ class EdgeTripletIteratorSuite extends FunSuite { test("iterator.toList") { - val builder = new EdgePartitionBuilder[Int] + val builder = new EdgePartitionBuilder[Int, Int] builder.add(1, 2, 0) builder.add(1, 3, 0) builder.add(1, 4, 0) - val vidmap = new VertexIdToIndexMap - vidmap.add(1) - vidmap.add(2) - vidmap.add(3) - vidmap.add(4) - val vs = Array.fill(vidmap.capacity)(0) - val iter = new EdgeTripletIterator[Int, Int](vidmap, vs, builder.toEdgePartition) + val iter = new EdgeTripletIterator[Int, Int](builder.toEdgePartition, true, true) val result = iter.toList.map(et => (et.srcId, et.dstId)) assert(result === Seq((1, 2), (1, 3), (1, 4))) } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala index a048d13fd12b8..8bf1384d514c1 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala @@ -30,17 +30,6 @@ class VertexPartitionSuite extends FunSuite { assert(!vp.isDefined(-1)) } - test("isActive, numActives, replaceActives") { - val vp = VertexPartition(Iterator((0L, 1), (1L, 1))) - .filter { (vid, attr) => vid == 0 } - .replaceActives(Iterator(0, 2, 0)) - assert(vp.isActive(0)) - assert(!vp.isActive(1)) - assert(vp.isActive(2)) - assert(!vp.isActive(-1)) - assert(vp.numActives == Some(2)) - } - test("map") { val vp = VertexPartition(Iterator((0L, 1), (1L, 1))).map { (vid, attr) => 2 } assert(vp(0) === 2) diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index efdb38e907d14..fafc9b36a77d3 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -76,6 +76,8 @@ object MimaBuild { excludeSparkClass("util.XORShiftRandom") ++ excludeSparkClass("graphx.EdgeRDD") ++ excludeSparkClass("graphx.VertexRDD") ++ + excludeSparkClass("graphx.impl.GraphImpl") ++ + excludeSparkClass("graphx.impl.RoutingTable") ++ excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ excludeSparkClass("mllib.optimization.SquaredGradient") ++ excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ From 2b7bd29eb6ee5baf739eec143044ecfc296b9b1f Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 10 May 2014 20:50:40 -0700 Subject: [PATCH 306/641] SPARK-1789. Multiple versions of Netty dependencies cause FlumeStreamSuite failure TL;DR is there is a bit of JAR hell trouble with Netty, that can be mostly resolved and will resolve a test failure. I hit the error described at http://apache-spark-user-list.1001560.n3.nabble.com/SparkContext-startup-time-out-td1753.html while running FlumeStreamingSuite, and have for a short while (is it just me?) velvia notes: "I have found a workaround. If you add akka 2.2.4 to your dependencies, then everything works, probably because akka 2.2.4 brings in newer version of Jetty." There are at least 3 versions of Netty in play in the build: - the new Flume 1.4.0 dependency brings in io.netty:netty:3.4.0.Final, and that is the immediate problem - the custom version of akka 2.2.3 depends on io.netty:netty:3.6.6. - but, Spark Core directly uses io.netty:netty-all:4.0.17.Final The POMs try to exclude other versions of netty, but are excluding org.jboss.netty:netty, when in fact older versions of io.netty:netty (not netty-all) are also an issue. The org.jboss.netty:netty excludes are largely unnecessary. I replaced many of them with io.netty:netty exclusions until everything agreed on io.netty:netty-all:4.0.17.Final. But this didn't work, since Akka 2.2.3 doesn't work with Netty 4.x. Down-grading to 3.6.6.Final across the board made some Spark code not compile. If the build *keeps* io.netty:netty:3.6.6.Final as well, everything seems to work. Part of the reason seems to be that Netty 3.x used the old `org.jboss.netty` packages. This is less than ideal, but is no worse than the current situation. So this PR resolves the issue and improves the JAR hell, even if it leaves the existing theoretical Netty 3-vs-4 conflict: - Remove org.jboss.netty excludes where possible, for clarity; they're not needed except with Hadoop artifacts - Add io.netty:netty excludes where needed -- except, let akka keep its io.netty:netty - Change a bit of test code that actually depended on Netty 3.x, to use 4.x equivalent - Update SBT build accordingly A better change would be to update Akka far enough such that it agrees on Netty 4.x, but I don't know if that's feasible. Author: Sean Owen Closes #723 from srowen/SPARK-1789 and squashes the following commits: 43661b7 [Sean Owen] Update and add Netty excludes to prevent some JAR conflicts that cause test issues --- .../org/apache/spark/LocalSparkContext.scala | 3 +- examples/pom.xml | 4 +++ external/flume/pom.xml | 2 +- external/mqtt/pom.xml | 6 ---- external/twitter/pom.xml | 6 ---- external/zeromq/pom.xml | 6 ---- pom.xml | 32 ----------------- project/SparkBuild.scala | 35 ++++++++++--------- 8 files changed, 24 insertions(+), 70 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala index 4b972f88a9542..53e367a61715b 100644 --- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -17,8 +17,7 @@ package org.apache.spark -import org.jboss.netty.logging.InternalLoggerFactory -import org.jboss.netty.logging.Slf4JLoggerFactory +import _root_.io.netty.util.internal.logging.{Slf4JLoggerFactory, InternalLoggerFactory} import org.scalatest.BeforeAndAfterAll import org.scalatest.BeforeAndAfterEach import org.scalatest.Suite diff --git a/examples/pom.xml b/examples/pom.xml index e1fc149d87f17..874bcd7916f35 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -106,6 +106,10 @@ org.jboss.netty netty + + io.netty + netty + commons-logging commons-logging diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 03d3b2394f510..6aec215687fe0 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -50,7 +50,7 @@ 1.4.0 - org.jboss.netty + io.netty netty diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 9aa1c1a9f5b80..7b2dc5ba1d7f9 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -53,12 +53,6 @@ ${akka.group} akka-zeromq_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - org.scalatest diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index a443459594710..5766d3a0d44ec 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -48,12 +48,6 @@ org.twitter4j twitter4j-stream 3.0.3 - - - org.jboss.netty - netty - - org.scalatest diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index a40e55876e640..4ed4196bd8662 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -48,12 +48,6 @@ ${akka.group} akka-zeromq_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - org.scalatest diff --git a/pom.xml b/pom.xml index e0bff60a54cde..c4e1c6be52a1b 100644 --- a/pom.xml +++ b/pom.xml @@ -324,45 +324,21 @@ ${akka.group} akka-actor_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - ${akka.group} akka-remote_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - ${akka.group} akka-slf4j_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - ${akka.group} akka-testkit_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - colt @@ -513,10 +489,6 @@ avro ${avro.version} - - org.jboss.netty - netty - io.netty netty @@ -551,10 +523,6 @@ avro-mapred ${avro.version} - - org.jboss.netty - netty - io.netty netty diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index af882b3ea7beb..a12c61853e410 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -303,7 +303,8 @@ object SparkBuild extends Build { val parquetVersion = "1.4.3" val slf4jVersion = "1.7.5" - val excludeNetty = ExclusionRule(organization = "org.jboss.netty") + val excludeJBossNetty = ExclusionRule(organization = "org.jboss.netty") + val excludeIONetty = ExclusionRule(organization = "io.netty") val excludeEclipseJetty = ExclusionRule(organization = "org.eclipse.jetty") val excludeAsm = ExclusionRule(organization = "org.ow2.asm") val excludeOldAsm = ExclusionRule(organization = "asm") @@ -337,8 +338,8 @@ object SparkBuild extends Build { "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407 "com.ning" % "compress-lzf" % "1.0.0", "org.xerial.snappy" % "snappy-java" % "1.0.5", - "org.spark-project.akka" %% "akka-remote" % akkaVersion excludeAll(excludeNetty), - "org.spark-project.akka" %% "akka-slf4j" % akkaVersion excludeAll(excludeNetty), + "org.spark-project.akka" %% "akka-remote" % akkaVersion, + "org.spark-project.akka" %% "akka-slf4j" % akkaVersion, "org.spark-project.akka" %% "akka-testkit" % akkaVersion % "test", "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "colt" % "colt" % "1.2.0", @@ -346,8 +347,8 @@ object SparkBuild extends Build { "commons-net" % "commons-net" % "2.2", "net.java.dev.jets3t" % "jets3t" % jets3tVersion excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", - "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm), - "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeNetty), + "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm), + "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeJBossNetty), "com.codahale.metrics" % "metrics-core" % codahaleMetricsVersion, "com.codahale.metrics" % "metrics-jvm" % codahaleMetricsVersion, "com.codahale.metrics" % "metrics-json" % codahaleMetricsVersion, @@ -421,7 +422,7 @@ object SparkBuild extends Build { v => "spark-examples-" + v + "-hadoop" + hadoopVersion + ".jar" }, libraryDependencies ++= Seq( "com.twitter" %% "algebird-core" % "0.1.11", - "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging, excludeJruby), + "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeIONetty, excludeJBossNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging, excludeJruby), "org.apache.cassandra" % "cassandra-all" % "1.2.6" exclude("com.google.guava", "guava") exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru") @@ -429,7 +430,7 @@ object SparkBuild extends Build { exclude("io.netty", "netty") exclude("jline","jline") exclude("org.apache.cassandra.deps", "avro") - excludeAll(excludeSLF4J), + excludeAll(excludeSLF4J, excludeIONetty), "com.github.scopt" %% "scopt" % "3.2.0" ) ) ++ assemblySettings ++ extraAssemblySettings @@ -561,11 +562,11 @@ object SparkBuild extends Build { def yarnEnabledSettings = Seq( libraryDependencies ++= Seq( // Exclude rule required for all ? - "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-server-web-proxy" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm) + "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-server-web-proxy" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm) ) ) @@ -593,7 +594,7 @@ object SparkBuild extends Build { name := "spark-streaming-twitter", previousArtifact := sparkPreviousArtifact("spark-streaming-twitter"), libraryDependencies ++= Seq( - "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty) + "org.twitter4j" % "twitter4j-stream" % "3.0.3" ) ) @@ -601,12 +602,12 @@ object SparkBuild extends Build { name := "spark-streaming-kafka", previousArtifact := sparkPreviousArtifact("spark-streaming-kafka"), libraryDependencies ++= Seq( - "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), + "com.github.sgroschupf" % "zkclient" % "0.1", "org.apache.kafka" %% "kafka" % "0.8.0" exclude("com.sun.jdmk", "jmxtools") exclude("com.sun.jmx", "jmxri") exclude("net.sf.jopt-simple", "jopt-simple") - excludeAll(excludeNetty, excludeSLF4J) + excludeAll(excludeSLF4J) ) ) @@ -614,7 +615,7 @@ object SparkBuild extends Build { name := "spark-streaming-flume", previousArtifact := sparkPreviousArtifact("spark-streaming-flume"), libraryDependencies ++= Seq( - "org.apache.flume" % "flume-ng-sdk" % "1.4.0" % "compile" excludeAll(excludeNetty, excludeThrift) + "org.apache.flume" % "flume-ng-sdk" % "1.4.0" % "compile" excludeAll(excludeIONetty, excludeThrift) ) ) @@ -622,7 +623,7 @@ object SparkBuild extends Build { name := "spark-streaming-zeromq", previousArtifact := sparkPreviousArtifact("spark-streaming-zeromq"), libraryDependencies ++= Seq( - "org.spark-project.akka" %% "akka-zeromq" % akkaVersion excludeAll(excludeNetty) + "org.spark-project.akka" %% "akka-zeromq" % akkaVersion ) ) From 83e0424d87022e7a967088365931a08aa06ffd9f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 10 May 2014 20:58:02 -0700 Subject: [PATCH 307/641] [SPARK-1774] Respect SparkSubmit --jars on YARN (client) SparkSubmit ignores `--jars` for YARN client. This is a bug. This PR also automatically adds the application jar to `spark.jar`. Previously, when running as yarn-client, you must specify the jar additionally through `--files` (because `--jars` didn't work). Now you don't have to explicitly specify it through either. Tested on a YARN cluster. Author: Andrew Or Closes #710 from andrewor14/yarn-jars and squashes the following commits: 35d1928 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-jars c27bf6c [Andrew Or] For yarn-cluster and python, do not add primaryResource to spark.jar c92c5bf [Andrew Or] Minor cleanups 269f9f3 [Andrew Or] Fix format 013d840 [Andrew Or] Fix tests 1407474 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-jars 3bb75e8 [Andrew Or] Allow SparkSubmit --jars to take effect in yarn-client mode --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 39 ++++--- .../spark/deploy/SparkSubmitSuite.scala | 110 ++++++++++++------ .../spark/deploy/yarn/ClientArguments.scala | 4 +- 4 files changed, 102 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c639b3e15ded5..71bab295442fc 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -917,7 +917,7 @@ class SparkContext(config: SparkConf) extends Logging { if (SparkHadoopUtil.get.isYarnMode() && (master == "yarn-standalone" || master == "yarn-cluster")) { // In order for this to work in yarn-cluster mode the user must specify the - // --addjars option to the client to upload the file into the distributed cache + // --addJars option to the client to upload the file into the distributed cache // of the AM to make it show up in the current working directory. val fileName = new Path(uri.getPath).getName() try { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 16de6f7cdb100..c6d3cbd2e728b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -67,8 +67,7 @@ object SparkSubmit { private[spark] def printWarning(str: String) = printStream.println("Warning: " + str) /** - * @return - * a tuple containing the arguments for the child, a list of classpath + * @return a tuple containing the arguments for the child, a list of classpath * entries for the child, a list of system propertes, a list of env vars * and the main class for the child */ @@ -115,13 +114,16 @@ object SparkSubmit { val sysProps = new HashMap[String, String]() var childMainClass = "" + val isPython = args.isPython + val isYarnCluster = clusterManager == YARN && deployOnCluster + if (clusterManager == MESOS && deployOnCluster) { printErrorAndExit("Cannot currently run driver on the cluster in Mesos") } // If we're running a Python app, set the Java class to run to be our PythonRunner, add // Python files to deployment list, and pass the main file and Python path to PythonRunner - if (args.isPython) { + if (isPython) { if (deployOnCluster) { printErrorAndExit("Cannot currently run Python driver programs on cluster") } @@ -161,6 +163,7 @@ object SparkSubmit { val options = List[OptionAssigner]( OptionAssigner(args.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), OptionAssigner(args.name, ALL_CLUSTER_MGRS, false, sysProp = "spark.app.name"), + OptionAssigner(args.name, YARN, true, clOption = "--name", sysProp = "spark.app.name"), OptionAssigner(args.driverExtraClassPath, STANDALONE | YARN, true, sysProp = "spark.driver.extraClassPath"), OptionAssigner(args.driverExtraJavaOptions, STANDALONE | YARN, true, @@ -168,7 +171,8 @@ object SparkSubmit { OptionAssigner(args.driverExtraLibraryPath, STANDALONE | YARN, true, sysProp = "spark.driver.extraLibraryPath"), OptionAssigner(args.driverMemory, YARN, true, clOption = "--driver-memory"), - OptionAssigner(args.name, YARN, true, clOption = "--name", sysProp = "spark.app.name"), + OptionAssigner(args.driverMemory, STANDALONE, true, clOption = "--memory"), + OptionAssigner(args.driverCores, STANDALONE, true, clOption = "--cores"), OptionAssigner(args.queue, YARN, true, clOption = "--queue"), OptionAssigner(args.queue, YARN, false, sysProp = "spark.yarn.queue"), OptionAssigner(args.numExecutors, YARN, true, clOption = "--num-executors"), @@ -176,20 +180,18 @@ object SparkSubmit { OptionAssigner(args.executorMemory, YARN, true, clOption = "--executor-memory"), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, false, sysProp = "spark.executor.memory"), - OptionAssigner(args.driverMemory, STANDALONE, true, clOption = "--memory"), - OptionAssigner(args.driverCores, STANDALONE, true, clOption = "--cores"), OptionAssigner(args.executorCores, YARN, true, clOption = "--executor-cores"), OptionAssigner(args.executorCores, YARN, false, sysProp = "spark.executor.cores"), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, false, sysProp = "spark.cores.max"), OptionAssigner(args.files, YARN, false, sysProp = "spark.yarn.dist.files"), OptionAssigner(args.files, YARN, true, clOption = "--files"), + OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.files"), + OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), OptionAssigner(args.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), OptionAssigner(args.archives, YARN, true, clOption = "--archives"), OptionAssigner(args.jars, YARN, true, clOption = "--addJars"), - OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.files"), - OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), - OptionAssigner(args.jars, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.jars") + OptionAssigner(args.jars, ALL_CLUSTER_MGRS, false, sysProp = "spark.jars") ) // For client mode make any added jars immediately visible on the classpath @@ -212,9 +214,10 @@ object SparkSubmit { } } - // For standalone mode, add the application jar automatically so the user doesn't have to - // call sc.addJar. TODO: Standalone mode in the cluster - if (clusterManager == STANDALONE) { + // Add the application jar automatically so the user doesn't have to call sc.addJar + // For YARN cluster mode, the jar is already distributed on each node as "app.jar" + // For python files, the primary resource is already distributed as a regular file + if (!isYarnCluster && !isPython) { var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq()) if (args.primaryResource != RESERVED_JAR_NAME) { jars = jars ++ Seq(args.primaryResource) @@ -222,11 +225,11 @@ object SparkSubmit { sysProps.put("spark.jars", jars.mkString(",")) } + // Standalone cluster specific configurations if (deployOnCluster && clusterManager == STANDALONE) { if (args.supervise) { childArgs += "--supervise" } - childMainClass = "org.apache.spark.deploy.Client" childArgs += "launch" childArgs += (args.master, args.primaryResource, args.mainClass) @@ -243,6 +246,7 @@ object SparkSubmit { } } + // Read from default spark properties, if any for ((k, v) <- args.getDefaultSparkProperties) { if (!sysProps.contains(k)) sysProps(k) = v } @@ -250,9 +254,12 @@ object SparkSubmit { (childArgs, childClasspath, sysProps, childMainClass) } - private def launch(childArgs: ArrayBuffer[String], childClasspath: ArrayBuffer[String], - sysProps: Map[String, String], childMainClass: String, verbose: Boolean = false) - { + private def launch( + childArgs: ArrayBuffer[String], + childClasspath: ArrayBuffer[String], + sysProps: Map[String, String], + childMainClass: String, + verbose: Boolean = false) { if (verbose) { printStream.println(s"Main class:\n$childMainClass") printStream.println(s"Arguments:\n${childArgs.mkString("\n")}") diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index c9edb03cdeb0f..6c0deede53784 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -87,25 +87,41 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles arguments with --key=val") { - val clArgs = Seq("--jars=one.jar,two.jar,three.jar", "--name=myApp") + val clArgs = Seq( + "--jars=one.jar,two.jar,three.jar", + "--name=myApp") val appArgs = new SparkSubmitArguments(clArgs) appArgs.jars should be ("one.jar,two.jar,three.jar") appArgs.name should be ("myApp") } test("handles arguments to user program") { - val clArgs = Seq("--name", "myApp", "--class", "Foo", "userjar.jar", "some", "--weird", "args") + val clArgs = Seq( + "--name", "myApp", + "--class", "Foo", + "userjar.jar", + "some", + "--weird", "args") val appArgs = new SparkSubmitArguments(clArgs) appArgs.childArgs should be (Seq("some", "--weird", "args")) } test("handles YARN cluster mode") { - val clArgs = Seq("--deploy-mode", "cluster", - "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", - "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", - "--driver-memory", "4g", "--queue", "thequeue", "--files", "file1.txt,file2.txt", - "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", "--name", "beauty", - "thejar.jar", "arg1", "arg2") + val clArgs = Seq( + "--deploy-mode", "cluster", + "--master", "yarn", + "--executor-memory", "5g", + "--executor-cores", "5", + "--class", "org.SomeClass", + "--jars", "one.jar,two.jar,three.jar", + "--driver-memory", "4g", + "--queue", "thequeue", + "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", + "--num-executors", "6", + "--name", "beauty", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") @@ -127,12 +143,21 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles YARN client mode") { - val clArgs = Seq("--deploy-mode", "client", - "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", - "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", - "--driver-memory", "4g", "--queue", "thequeue", "--files", "file1.txt,file2.txt", - "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", "--name", "trill", - "thejar.jar", "arg1", "arg2") + val clArgs = Seq( + "--deploy-mode", "client", + "--master", "yarn", + "--executor-memory", "5g", + "--executor-cores", "5", + "--class", "org.SomeClass", + "--jars", "one.jar,two.jar,three.jar", + "--driver-memory", "4g", + "--queue", "thequeue", + "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", + "--num-executors", "6", + "--name", "trill", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -142,6 +167,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { classpath should contain ("two.jar") classpath should contain ("three.jar") sysProps("spark.app.name") should be ("trill") + sysProps("spark.jars") should be ("one.jar,two.jar,three.jar,thejar.jar") sysProps("spark.executor.memory") should be ("5g") sysProps("spark.executor.cores") should be ("5") sysProps("spark.yarn.queue") should be ("thequeue") @@ -152,9 +178,15 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles standalone cluster mode") { - val clArgs = Seq("--deploy-mode", "cluster", - "--master", "spark://h:p", "--class", "org.SomeClass", - "--supervise", "--driver-memory", "4g", "--driver-cores", "5", "thejar.jar", "arg1", "arg2") + val clArgs = Seq( + "--deploy-mode", "cluster", + "--master", "spark://h:p", + "--class", "org.SomeClass", + "--supervise", + "--driver-memory", "4g", + "--driver-cores", "5", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") @@ -166,9 +198,15 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles standalone client mode") { - val clArgs = Seq("--deploy-mode", "client", - "--master", "spark://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", - "--class", "org.SomeClass", "--driver-memory", "4g", "thejar.jar", "arg1", "arg2") + val clArgs = Seq( + "--deploy-mode", "client", + "--master", "spark://h:p", + "--executor-memory", "5g", + "--total-executor-cores", "5", + "--class", "org.SomeClass", + "--driver-memory", "4g", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -179,9 +217,15 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles mesos client mode") { - val clArgs = Seq("--deploy-mode", "client", - "--master", "mesos://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", - "--class", "org.SomeClass", "--driver-memory", "4g", "thejar.jar", "arg1", "arg2") + val clArgs = Seq( + "--deploy-mode", "client", + "--master", "mesos://h:p", + "--executor-memory", "5g", + "--total-executor-cores", "5", + "--class", "org.SomeClass", + "--driver-memory", "4g", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -192,15 +236,17 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("launch simple application with spark-submit") { - runSparkSubmit( - Seq( - "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), - "--name", "testApp", - "--master", "local", - "unUsed.jar")) + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val args = Seq( + "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local", + unusedJar.toString) + runSparkSubmit(args) } test("spark submit includes jars passed in through --jar") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) val jarsString = Seq(jar1, jar2).map(j => j.toString).mkString(",") @@ -209,7 +255,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { "--name", "testApp", "--master", "local-cluster[2,1,512]", "--jars", jarsString, - "unused.jar") + unusedJar.toString) runSparkSubmit(args) } @@ -227,7 +273,7 @@ object JarCreationTest { def main(args: Array[String]) { val conf = new SparkConf() val sc = new SparkContext(conf) - val result = sc.makeRDD(1 to 100, 10).mapPartitions{ x => + val result = sc.makeRDD(1 to 100, 10).mapPartitions { x => var foundClasses = false try { Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) @@ -248,7 +294,6 @@ object SimpleApplicationTest { def main(args: Array[String]) { val conf = new SparkConf() val sc = new SparkContext(conf) - val configs = Seq("spark.master", "spark.app.name") for (config <- configs) { val masterValue = conf.get(config) @@ -266,6 +311,5 @@ object SimpleApplicationTest { s"Master had $config=$masterValue but executor had $config=$executorValue") } } - } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 3e4c739e34fe9..b2c413b6d267c 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.yarn import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.spark.SparkConf -import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo} +import org.apache.spark.scheduler.InputFormatInfo import org.apache.spark.util.IntParam import org.apache.spark.util.MemoryParam @@ -40,9 +40,7 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { var amMemory: Int = 512 // MB var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" - // TODO var inputFormatInfo: List[InputFormatInfo] = null - // TODO(harvey) var priority = 0 parseArgs(args.toList) From 70bcdef48a051028598d380d41dfce1c9bfb2b9b Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sat, 10 May 2014 21:08:04 -0700 Subject: [PATCH 308/641] Enabled incremental build that comes with sbt 0.13.2 More info at. https://github.com/sbt/sbt/issues/1010 Author: Prashant Sharma Closes #525 from ScrapCodes/sbt-inc-opt and squashes the following commits: ba8fa42 [Prashant Sharma] Enabled incremental build that comes with sbt 0.13.2 --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a12c61853e410..12791e490ae2b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -176,7 +176,7 @@ object SparkBuild extends Build { retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", transitiveClassifiers in Scope.GlobalScope := Seq("sources"), testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), - + incOptions := incOptions.value.withNameHashing(true), // Fork new JVMs for tests and set Java options for those fork := true, javaOptions in Test += "-Dspark.home=" + sparkHome, From 6bee01dd04ef73c6b829110ebcdd622d521ea8ff Mon Sep 17 00:00:00 2001 From: witgo Date: Sun, 11 May 2014 14:34:27 -0700 Subject: [PATCH 309/641] remove outdated runtime Information scala home Author: witgo Closes #728 from witgo/scala_home and squashes the following commits: cdfd8be [witgo] Merge branch 'master' of https://github.com/apache/spark into scala_home fac094a [witgo] remove outdated runtime Information scala home --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 806e77d98fc5f..19d507c0cf860 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -281,8 +281,7 @@ object SparkEnv extends Logging { val jvmInformation = Seq( ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), ("Java Home", Properties.javaHome), - ("Scala Version", Properties.versionString), - ("Scala Home", Properties.scalaHome) + ("Scala Version", Properties.versionString) ).sorted // Spark properties From 7d9cc9214bd06495f6838e355331dd2b5f1f7407 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 11 May 2014 17:11:55 -0700 Subject: [PATCH 310/641] SPARK-1770: Load balance elements when repartitioning. This patch adds better balancing when performing a repartition of an RDD. Previously the elements in the RDD were hash partitioned, meaning if the RDD was skewed certain partitions would end up being very large. This commit adds load balancing of elements across the repartitioned RDD splits. The load balancing is not perfect: a given output partition can have up to N more elements than the average if there are N input partitions. However, some randomization is used to minimize the probabiliy that this happens. Author: Patrick Wendell Closes #727 from pwendell/load-balance and squashes the following commits: f9da752 [Patrick Wendell] Response to Matei's feedback acfa46a [Patrick Wendell] SPARK-1770: Load balance elements when repartitioning. --- .../main/scala/org/apache/spark/rdd/RDD.scala | 15 +++++++-- .../scala/org/apache/spark/rdd/RDDSuite.scala | 33 +++++++++++++++++++ 2 files changed, 46 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index a1ca612cc9a09..aa03e9276fb34 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -328,11 +328,22 @@ abstract class RDD[T: ClassTag]( def coalesce(numPartitions: Int, shuffle: Boolean = false)(implicit ord: Ordering[T] = null) : RDD[T] = { if (shuffle) { + /** Distributes elements evenly across output partitions, starting from a random partition. */ + def distributePartition(index: Int, items: Iterator[T]): Iterator[(Int, T)] = { + var position = (new Random(index)).nextInt(numPartitions) + items.map { t => + // Note that the hash code of the key will just be the key itself. The HashPartitioner + // will mod it with the number of total partitions. + position = position + 1 + (position, t) + } + } + // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( - new ShuffledRDD[T, Null, (T, Null)](map(x => (x, null)), + new ShuffledRDD[Int, T, (Int, T)](mapPartitionsWithIndex(distributePartition), new HashPartitioner(numPartitions)), - numPartitions).keys + numPartitions).values } else { new CoalescedRDD(this, numPartitions) } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 8da9a0da700e0..e686068f7a99a 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -202,6 +202,39 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(repartitioned2.collect().toSet === (1 to 1000).toSet) } + test("repartitioned RDDs perform load balancing") { + // Coalesce partitions + val input = Array.fill(1000)(1) + val initialPartitions = 10 + val data = sc.parallelize(input, initialPartitions) + + val repartitioned1 = data.repartition(2) + assert(repartitioned1.partitions.size == 2) + val partitions1 = repartitioned1.glom().collect() + // some noise in balancing is allowed due to randomization + assert(math.abs(partitions1(0).length - 500) < initialPartitions) + assert(math.abs(partitions1(1).length - 500) < initialPartitions) + assert(repartitioned1.collect() === input) + + def testSplitPartitions(input: Seq[Int], initialPartitions: Int, finalPartitions: Int) { + val data = sc.parallelize(input, initialPartitions) + val repartitioned = data.repartition(finalPartitions) + assert(repartitioned.partitions.size === finalPartitions) + val partitions = repartitioned.glom().collect() + // assert all elements are present + assert(repartitioned.collect().sortWith(_ > _).toSeq === input.toSeq.sortWith(_ > _).toSeq) + // assert no bucket is overloaded + for (partition <- partitions) { + val avg = input.size / finalPartitions + val maxPossible = avg + initialPartitions + assert(partition.length <= maxPossible) + } + } + + testSplitPartitions(Array.fill(100)(1), 10, 20) + testSplitPartitions(Array.fill(10000)(1) ++ Array.fill(10000)(2), 20, 100) + } + test("coalesced RDDs") { val data = sc.parallelize(1 to 10, 10) From 05c9aa9eb1b7f13cd40bbca23e6bc7e1d20e91cd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 11 May 2014 18:17:34 -0700 Subject: [PATCH 311/641] SPARK-1652: Set driver memory correctly in spark-submit. The previous check didn't account for the fact that the default deploy mode is "client" unless otherwise specified. Also, this sets the more narrowly defined SPARK_DRIVER_MEMORY instead of setting SPARK_MEM. Author: Patrick Wendell Closes #730 from pwendell/spark-submit and squashes the following commits: 430b98f [Patrick Wendell] Feedback from Aaron e788edf [Patrick Wendell] Changes based on Aaron's feedback f508146 [Patrick Wendell] SPARK-1652: Set driver memory correctly in spark-submit. --- bin/spark-submit | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/bin/spark-submit b/bin/spark-submit index 49bc26252cadf..63903b17a2902 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -35,8 +35,10 @@ while (($#)); do shift done -if [ ! -z $DRIVER_MEMORY ] && [ ! -z $DEPLOY_MODE ] && [ $DEPLOY_MODE = "client" ]; then - export SPARK_MEM=$DRIVER_MEMORY +DEPLOY_MODE=${DEPLOY_MODE:-"client"} + +if [ -n "$DRIVER_MEMORY" ] && [ $DEPLOY_MODE == "client" ]; then + export SPARK_DRIVER_MEMORY=$DRIVER_MEMORY fi $SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" From f938a155b2a9c126b292d5403aca31de83d5105a Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 11 May 2014 18:33:46 -0700 Subject: [PATCH 312/641] Fix error in 2d Graph Partitioner Their was a minor bug in which negative partition ids could be generated when constructing a 2D partitioning of a graph. This could lead to an inefficient 2D partition for large vertex id values. Author: Joseph E. Gonzalez Closes #709 from jegonzal/fix_2d_partitioning and squashes the following commits: 937c562 [Joseph E. Gonzalez] fixing bug in 2d partitioning algorithm where negative partition ids could be generated. --- .../scala/org/apache/spark/graphx/PartitionStrategy.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala index 0470d74cf9efe..1526ccef06fd4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -78,8 +78,8 @@ object PartitionStrategy { override def getPartition(src: VertexId, dst: VertexId, numParts: PartitionID): PartitionID = { val ceilSqrtNumParts: PartitionID = math.ceil(math.sqrt(numParts)).toInt val mixingPrime: VertexId = 1125899906842597L - val col: PartitionID = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt - val row: PartitionID = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt + val col: PartitionID = (math.abs(src * mixingPrime) % ceilSqrtNumParts).toInt + val row: PartitionID = (math.abs(dst * mixingPrime) % ceilSqrtNumParts).toInt (col * ceilSqrtNumParts + row) % numParts } } From a6b02fb7486356493474c7f42bb714c9cce215ca Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sun, 11 May 2014 19:20:42 -0700 Subject: [PATCH 313/641] SPARK-1786: Edge Partition Serialization This appears to address the issue with edge partition serialization. The solution appears to be just registering the `PrimitiveKeyOpenHashMap`. However I noticed that we appear to have forked that code in GraphX but retained the same name (which is confusing). I also renamed our local copy to `GraphXPrimitiveKeyOpenHashMap`. We should consider dropping that and using the one in Spark if possible. Author: Ankur Dave Author: Joseph E. Gonzalez Closes #724 from jegonzal/edge_partition_serialization and squashes the following commits: b0a525a [Ankur Dave] Disable reference tracking to fix serialization test bb7f548 [Ankur Dave] Add failing test for EdgePartition Kryo serialization 67dac22 [Joseph E. Gonzalez] Making EdgePartition serializable. --- .../spark/graphx/GraphKryoRegistrator.scala | 9 ++++++--- .../spark/graphx/impl/EdgePartition.scala | 14 +++++++------- .../graphx/impl/EdgePartitionBuilder.scala | 4 ++-- .../graphx/impl/EdgeTripletIterator.scala | 2 +- .../graphx/impl/RoutingTablePartition.scala | 4 ++-- .../graphx/impl/ShippableVertexPartition.scala | 2 +- .../spark/graphx/impl/VertexPartition.scala | 2 +- .../graphx/impl/VertexPartitionBase.scala | 6 +++--- .../graphx/impl/VertexPartitionBaseOps.scala | 4 ++-- ...ala => GraphXPrimitiveKeyOpenHashMap.scala} | 2 +- .../spark/graphx/impl/EdgePartitionSuite.scala | 18 ++++++++++++++++++ 11 files changed, 44 insertions(+), 23 deletions(-) rename graphx/src/main/scala/org/apache/spark/graphx/util/collection/{PrimitiveKeyOpenHashMap.scala => GraphXPrimitiveKeyOpenHashMap.scala} (98%) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index d295d0127ac72..f97f329c0e832 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -24,6 +24,9 @@ import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx.impl._ +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.OpenHashSet + /** * Registers GraphX classes with Kryo for improved performance. @@ -43,8 +46,8 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[PartitionStrategy]) kryo.register(classOf[BoundedPriorityQueue[Object]]) kryo.register(classOf[EdgeDirection]) - - // This avoids a large number of hash table lookups. - kryo.setReferences(false) + kryo.register(classOf[GraphXPrimitiveKeyOpenHashMap[VertexId, Int]]) + kryo.register(classOf[OpenHashSet[Int]]) + kryo.register(classOf[OpenHashSet[Long]]) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index 871e81f8d245c..a5c9cd1f8b4e6 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * A collection of edges stored in columnar format, along with any vertex attributes referenced. The @@ -42,12 +42,12 @@ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] class EdgePartition[ @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag, VD: ClassTag]( - @transient val srcIds: Array[VertexId], - @transient val dstIds: Array[VertexId], - @transient val data: Array[ED], - @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int], - @transient val vertices: VertexPartition[VD], - @transient val activeSet: Option[VertexSet] = None + val srcIds: Array[VertexId] = null, + val dstIds: Array[VertexId] = null, + val data: Array[ED] = null, + val index: GraphXPrimitiveKeyOpenHashMap[VertexId, Int] = null, + val vertices: VertexPartition[VD] = null, + val activeSet: Option[VertexSet] = None ) extends Serializable { /** Return a new `EdgePartition` with the specified edge data. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index ecb49bef42e45..4520beb991515 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -23,7 +23,7 @@ import scala.util.Sorting import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: ClassTag]( @@ -41,7 +41,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla val srcIds = new Array[VertexId](edgeArray.size) val dstIds = new Array[VertexId](edgeArray.size) val data = new Array[ED](edgeArray.size) - val index = new PrimitiveKeyOpenHashMap[VertexId, Int] + val index = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int] // Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and // adding them to the index if (edgeArray.length > 0) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index ebb0b9418d65d..56f79a7097fce 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * The Iterator type returned when constructing edge triplets. This could be an anonymous class in diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index 927e32ad0f448..d02e9238adba5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * A message from the edge partition `pid` to the vertex partition containing `vid` specifying that @@ -69,7 +69,7 @@ object RoutingTablePartition { : Iterator[RoutingTableMessage] = { // Determine which positions each vertex id appears in using a map where the low 2 bits // represent src and dst - val map = new PrimitiveKeyOpenHashMap[VertexId, Byte] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, Byte] edgePartition.srcIds.iterator.foreach { srcId => map.changeValue(srcId, 0x1, (b: Byte) => (b | 0x1).toByte) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index f4e221d4e05ae..dca54b8a7da86 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** Stores vertex attributes to ship to an edge partition. */ private[graphx] diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index f1d174720a1ba..55c7a19d1bdab 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] object VertexPartition { /** Construct a `VertexPartition` from the given vertices. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala index 8d9e0204d27f2..34939b24440aa 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] object VertexPartitionBase { /** @@ -32,7 +32,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)]) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map(pair._1) = pair._2 } @@ -45,7 +45,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map.setMerge(pair._1, pair._2, mergeFunc) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index 21ff615feca6c..a4f769b294010 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -25,7 +25,7 @@ import org.apache.spark.Logging import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * An class containing additional operations for subclasses of VertexPartitionBase that provide @@ -224,7 +224,7 @@ private[graphx] abstract class VertexPartitionBaseOps * Construct a new VertexPartition whose index contains only the vertices in the mask. */ def reindex(): Self[VD] = { - val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] + val hashMap = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] val arbitraryMerge = (a: VD, b: VD) => a for ((k, v) <- self.iterator) { hashMap.setMerge(k, v, arbitraryMerge) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala similarity index 98% rename from graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala index 7b02e2ed1a9cb..57b01b6f2e1fb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala @@ -29,7 +29,7 @@ import scala.reflect._ * Under the hood, it uses our OpenHashSet implementation. */ private[graphx] -class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, +class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, @specialized(Long, Int, Double) V: ClassTag]( val keySet: OpenHashSet[K], var _values: Array[V]) extends Iterable[(K, V)] diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index d2e0c01bc35ef..28fd112f2b124 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -22,6 +22,9 @@ import scala.util.Random import org.scalatest.FunSuite +import org.apache.spark.SparkConf +import org.apache.spark.serializer.KryoSerializer + import org.apache.spark.graphx._ class EdgePartitionSuite extends FunSuite { @@ -120,4 +123,19 @@ class EdgePartitionSuite extends FunSuite { assert(!ep.isActive(-1)) assert(ep.numActives == Some(2)) } + + test("Kryo serialization") { + val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) + val a: EdgePartition[Int, Int] = makeEdgePartition(aList) + val conf = new SparkConf() + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + val s = new KryoSerializer(conf).newInstance() + val aSer: EdgePartition[Int, Int] = s.deserialize(s.serialize(a)) + assert(aSer.srcIds.toList === a.srcIds.toList) + assert(aSer.dstIds.toList === a.dstIds.toList) + assert(aSer.data.toList === a.data.toList) + assert(aSer.index != null) + assert(aSer.vertices.iterator.toSet === a.vertices.iterator.toSet) + } } From af15c82bfe2c3f73142b8f310784a0e85841539d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 10:49:03 -0700 Subject: [PATCH 314/641] Revert "SPARK-1786: Edge Partition Serialization" This reverts commit a6b02fb7486356493474c7f42bb714c9cce215ca. --- .../spark/graphx/GraphKryoRegistrator.scala | 9 +++------ .../spark/graphx/impl/EdgePartition.scala | 14 +++++++------- .../graphx/impl/EdgePartitionBuilder.scala | 4 ++-- .../graphx/impl/EdgeTripletIterator.scala | 2 +- .../graphx/impl/RoutingTablePartition.scala | 4 ++-- .../graphx/impl/ShippableVertexPartition.scala | 2 +- .../spark/graphx/impl/VertexPartition.scala | 2 +- .../graphx/impl/VertexPartitionBase.scala | 6 +++--- .../graphx/impl/VertexPartitionBaseOps.scala | 4 ++-- ...Map.scala => PrimitiveKeyOpenHashMap.scala} | 2 +- .../spark/graphx/impl/EdgePartitionSuite.scala | 18 ------------------ 11 files changed, 23 insertions(+), 44 deletions(-) rename graphx/src/main/scala/org/apache/spark/graphx/util/collection/{GraphXPrimitiveKeyOpenHashMap.scala => PrimitiveKeyOpenHashMap.scala} (98%) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index f97f329c0e832..d295d0127ac72 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -24,9 +24,6 @@ import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx.impl._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap -import org.apache.spark.util.collection.OpenHashSet - /** * Registers GraphX classes with Kryo for improved performance. @@ -46,8 +43,8 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[PartitionStrategy]) kryo.register(classOf[BoundedPriorityQueue[Object]]) kryo.register(classOf[EdgeDirection]) - kryo.register(classOf[GraphXPrimitiveKeyOpenHashMap[VertexId, Int]]) - kryo.register(classOf[OpenHashSet[Int]]) - kryo.register(classOf[OpenHashSet[Long]]) + + // This avoids a large number of hash table lookups. + kryo.setReferences(false) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index a5c9cd1f8b4e6..871e81f8d245c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * A collection of edges stored in columnar format, along with any vertex attributes referenced. The @@ -42,12 +42,12 @@ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] class EdgePartition[ @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag, VD: ClassTag]( - val srcIds: Array[VertexId] = null, - val dstIds: Array[VertexId] = null, - val data: Array[ED] = null, - val index: GraphXPrimitiveKeyOpenHashMap[VertexId, Int] = null, - val vertices: VertexPartition[VD] = null, - val activeSet: Option[VertexSet] = None + @transient val srcIds: Array[VertexId], + @transient val dstIds: Array[VertexId], + @transient val data: Array[ED], + @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int], + @transient val vertices: VertexPartition[VD], + @transient val activeSet: Option[VertexSet] = None ) extends Serializable { /** Return a new `EdgePartition` with the specified edge data. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index 4520beb991515..ecb49bef42e45 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -23,7 +23,7 @@ import scala.util.Sorting import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: ClassTag]( @@ -41,7 +41,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla val srcIds = new Array[VertexId](edgeArray.size) val dstIds = new Array[VertexId](edgeArray.size) val data = new Array[ED](edgeArray.size) - val index = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int] + val index = new PrimitiveKeyOpenHashMap[VertexId, Int] // Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and // adding them to the index if (edgeArray.length > 0) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index 56f79a7097fce..ebb0b9418d65d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * The Iterator type returned when constructing edge triplets. This could be an anonymous class in diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index d02e9238adba5..927e32ad0f448 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * A message from the edge partition `pid` to the vertex partition containing `vid` specifying that @@ -69,7 +69,7 @@ object RoutingTablePartition { : Iterator[RoutingTableMessage] = { // Determine which positions each vertex id appears in using a map where the low 2 bits // represent src and dst - val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, Byte] + val map = new PrimitiveKeyOpenHashMap[VertexId, Byte] edgePartition.srcIds.iterator.foreach { srcId => map.changeValue(srcId, 0x1, (b: Byte) => (b | 0x1).toByte) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index dca54b8a7da86..f4e221d4e05ae 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** Stores vertex attributes to ship to an edge partition. */ private[graphx] diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index 55c7a19d1bdab..f1d174720a1ba 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] object VertexPartition { /** Construct a `VertexPartition` from the given vertices. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala index 34939b24440aa..8d9e0204d27f2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] object VertexPartitionBase { /** @@ -32,7 +32,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)]) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + val map = new PrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map(pair._1) = pair._2 } @@ -45,7 +45,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + val map = new PrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map.setMerge(pair._1, pair._2, mergeFunc) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index a4f769b294010..21ff615feca6c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -25,7 +25,7 @@ import org.apache.spark.Logging import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * An class containing additional operations for subclasses of VertexPartitionBase that provide @@ -224,7 +224,7 @@ private[graphx] abstract class VertexPartitionBaseOps * Construct a new VertexPartition whose index contains only the vertices in the mask. */ def reindex(): Self[VD] = { - val hashMap = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] val arbitraryMerge = (a: VD, b: VD) => a for ((k, v) <- self.iterator) { hashMap.setMerge(k, v, arbitraryMerge) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala similarity index 98% rename from graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala index 57b01b6f2e1fb..7b02e2ed1a9cb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala @@ -29,7 +29,7 @@ import scala.reflect._ * Under the hood, it uses our OpenHashSet implementation. */ private[graphx] -class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, +class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, @specialized(Long, Int, Double) V: ClassTag]( val keySet: OpenHashSet[K], var _values: Array[V]) extends Iterable[(K, V)] diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index 28fd112f2b124..d2e0c01bc35ef 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -22,9 +22,6 @@ import scala.util.Random import org.scalatest.FunSuite -import org.apache.spark.SparkConf -import org.apache.spark.serializer.KryoSerializer - import org.apache.spark.graphx._ class EdgePartitionSuite extends FunSuite { @@ -123,19 +120,4 @@ class EdgePartitionSuite extends FunSuite { assert(!ep.isActive(-1)) assert(ep.numActives == Some(2)) } - - test("Kryo serialization") { - val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) - val a: EdgePartition[Int, Int] = makeEdgePartition(aList) - val conf = new SparkConf() - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") - val s = new KryoSerializer(conf).newInstance() - val aSer: EdgePartition[Int, Int] = s.deserialize(s.serialize(a)) - assert(aSer.srcIds.toList === a.srcIds.toList) - assert(aSer.dstIds.toList === a.dstIds.toList) - assert(aSer.data.toList === a.data.toList) - assert(aSer.index != null) - assert(aSer.vertices.iterator.toSet === a.vertices.iterator.toSet) - } } From 3af1f386439cdddd42e545ad63d089f4dfdf9f8a Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 12 May 2014 11:08:52 -0700 Subject: [PATCH 315/641] SPARK-1772 Stop catching Throwable, let Executors die The main issue this patch fixes is [SPARK-1772](https://issues.apache.org/jira/browse/SPARK-1772), in which Executors may not die when fatal exceptions (e.g., OOM) are thrown. This patch causes Executors to delegate to the ExecutorUncaughtExceptionHandler when a fatal exception is thrown. This patch also continues the fight in the neverending war against `case t: Throwable =>`, by only catching Exceptions in many places, and adding a wrapper for Threads and Runnables to make sure any uncaught exceptions are at least printed to the logs. It also turns out that it is unlikely that the IndestructibleActorSystem actually works, given testing ([here](https://gist.github.com/aarondav/ca1f0cdcd50727f89c0d)). The uncaughtExceptionHandler is not called from the places that we expected it would be. [SPARK-1620](https://issues.apache.org/jira/browse/SPARK-1620) deals with part of this issue, but refactoring our Actor Systems to ensure that exceptions are dealt with properly is a much bigger change, outside the scope of this PR. Author: Aaron Davidson Closes #715 from aarondav/throwable and squashes the following commits: f9b9bfe [Aaron Davidson] Remove other redundant 'throw e' e937a0a [Aaron Davidson] Address Prashant and Matei's comments 1867867 [Aaron Davidson] [RFC] SPARK-1772 Stop catching Throwable, let Executors die --- .../org/apache/spark/ContextCleaner.scala | 11 +-- .../scala/org/apache/spark/SparkContext.scala | 12 ++-- .../apache/spark/api/python/PythonRDD.scala | 3 +- .../api/python/PythonWorkerFactory.scala | 1 - .../org/apache/spark/deploy/Client.scala | 2 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 2 +- .../spark/deploy/history/HistoryServer.scala | 8 +-- .../apache/spark/deploy/master/Master.scala | 4 +- .../spark/deploy/worker/DriverWrapper.scala | 2 +- .../CoarseGrainedExecutorBackend.scala | 2 +- .../org/apache/spark/executor/Executor.scala | 37 +++------- .../ExecutorUncaughtExceptionHandler.scala | 53 +++++++++++++++ .../scheduler/EventLoggingListener.scala | 4 +- .../spark/scheduler/TaskResultGetter.scala | 8 +-- .../spark/storage/DiskBlockManager.scala | 6 +- .../spark/storage/TachyonBlockManager.scala | 7 +- .../org/apache/spark/util/AkkaUtils.scala | 11 +-- .../util/IndestructibleActorSystem.scala | 68 ------------------- .../scala/org/apache/spark/util/Utils.scala | 26 ++++++- 19 files changed, 127 insertions(+), 140 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala delete mode 100644 core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 54e08d7866f75..e2d2250982daa 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils /** * Classes that represent cleaning tasks. @@ -110,7 +111,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Keep cleaning RDD, shuffle, and broadcast state. */ - private def keepCleaning() { + private def keepCleaning(): Unit = Utils.logUncaughtExceptions { while (!stopped) { try { val reference = Option(referenceQueue.remove(ContextCleaner.REF_QUEUE_POLL_TIMEOUT)) @@ -128,7 +129,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } } } catch { - case t: Throwable => logError("Error in cleaning thread", t) + case e: Exception => logError("Error in cleaning thread", e) } } } @@ -141,7 +142,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { listeners.foreach(_.rddCleaned(rddId)) logInfo("Cleaned RDD " + rddId) } catch { - case t: Throwable => logError("Error cleaning RDD " + rddId, t) + case e: Exception => logError("Error cleaning RDD " + rddId, e) } } @@ -154,7 +155,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { listeners.foreach(_.shuffleCleaned(shuffleId)) logInfo("Cleaned shuffle " + shuffleId) } catch { - case t: Throwable => logError("Error cleaning shuffle " + shuffleId, t) + case e: Exception => logError("Error cleaning shuffle " + shuffleId, e) } } @@ -166,7 +167,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { listeners.foreach(_.broadcastCleaned(broadcastId)) logInfo("Cleaned broadcast " + broadcastId) } catch { - case t: Throwable => logError("Error cleaning broadcast " + broadcastId, t) + case e: Exception => logError("Error cleaning broadcast " + broadcastId, e) } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 71bab295442fc..e6121a705497c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1494,8 +1494,8 @@ object SparkContext extends Logging { } catch { // TODO: Enumerate the exact reasons why it can fail // But irrespective of it, it means we cannot proceed ! - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) } } val backend = new CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) @@ -1510,8 +1510,8 @@ object SparkContext extends Logging { cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl] } catch { - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) } } @@ -1521,8 +1521,8 @@ object SparkContext extends Logging { val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext]) cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] } catch { - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 2971c277aa863..57b28b9972366 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -171,7 +171,7 @@ private[spark] class PythonRDD[T: ClassTag]( this.interrupt() } - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { try { SparkEnv.set(env) val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) @@ -282,7 +282,6 @@ private[spark] object PythonRDD { } } catch { case eof: EOFException => {} - case e: Throwable => throw e } JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 002f2acd94dee..759cbe2c46c52 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -71,7 +71,6 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String stopDaemon() startDaemon() new Socket(daemonHost, daemonPort) - case e: Throwable => throw e } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 7ead1171525d2..aeb159adc31d9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -157,7 +157,7 @@ object Client { // TODO: See if we can initialize akka so return messages are sent back using the same TCP // flow. Else, this (sadly) requires the DriverClient be routable from the Master. val (actorSystem, _) = AkkaUtils.createActorSystem( - "driverClient", Utils.localHostName(), 0, false, conf, new SecurityManager(conf)) + "driverClient", Utils.localHostName(), 0, conf, new SecurityManager(conf)) actorSystem.actorOf(Props(classOf[ClientActor], driverArgs, conf)) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index e2df1b8954124..148115d3ed351 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -103,7 +103,7 @@ object SparkHadoopUtil { .newInstance() .asInstanceOf[SparkHadoopUtil] } catch { - case th: Throwable => throw new SparkException("Unable to load YARN support", th) + case e: Exception => throw new SparkException("Unable to load YARN support", e) } } else { new SparkHadoopUtil diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 1238bbf9da2fd..a9c11dca5678e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -70,7 +70,7 @@ class HistoryServer( * TODO: Add a mechanism to update manually. */ private val logCheckingThread = new Thread { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { while (!stopped) { val now = System.currentTimeMillis if (now - lastLogCheckTime > UPDATE_INTERVAL_MS) { @@ -154,7 +154,7 @@ class HistoryServer( numCompletedApplications = logInfos.size } catch { - case t: Throwable => logError("Exception in checking for event log updates", t) + case e: Exception => logError("Exception in checking for event log updates", e) } } else { logWarning("Attempted to check for event log updates before binding the server.") @@ -231,8 +231,8 @@ class HistoryServer( dir.getModificationTime } } catch { - case t: Throwable => - logError("Exception in accessing modification time of %s".format(dir.getPath), t) + case e: Exception => + logError("Exception in accessing modification time of %s".format(dir.getPath), e) -1L } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index f254f5585ba25..c6dec305bffcb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -684,8 +684,8 @@ private[spark] class Master( webUi.attachSparkUI(ui) return true } catch { - case t: Throwable => - logError("Exception in replaying log for application %s (%s)".format(appName, app.id), t) + case e: Exception => + logError("Exception in replaying log for application %s (%s)".format(appName, app.id), e) } } else { logWarning("Application %s (%s) has no valid logs: %s".format(appName, app.id, eventLogDir)) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index be15138f62406..05e242e6df702 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -31,7 +31,7 @@ object DriverWrapper { case workerUrl :: mainClass :: extraArgs => val conf = new SparkConf() val (actorSystem, _) = AkkaUtils.createActorSystem("Driver", - Utils.localHostName(), 0, false, conf, new SecurityManager(conf)) + Utils.localHostName(), 0, conf, new SecurityManager(conf)) actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher") // Delegate to supplied main class diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index e912ae8a5d3c5..84aec65b7765d 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -105,7 +105,7 @@ private[spark] object CoarseGrainedExecutorBackend { // Create a new ActorSystem to run the backend, because we can't create a // SparkEnv / Executor before getting started with all our system properties, etc val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, - indestructible = true, conf = conf, new SecurityManager(conf)) + conf, new SecurityManager(conf)) // set it val sparkHostPort = hostname + ":" + boundPort actorSystem.actorOf( diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 98e7e0be813be..baee7a216a7c3 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -74,28 +74,7 @@ private[spark] class Executor( // Setup an uncaught exception handler for non-local mode. // Make any thread terminations due to uncaught exceptions kill the entire // executor process to avoid surprising stalls. - Thread.setDefaultUncaughtExceptionHandler( - new Thread.UncaughtExceptionHandler { - override def uncaughtException(thread: Thread, exception: Throwable) { - try { - logError("Uncaught exception in thread " + thread, exception) - - // We may have been called from a shutdown hook. If so, we must not call System.exit(). - // (If we do, we will deadlock.) - if (!Utils.inShutdown()) { - if (exception.isInstanceOf[OutOfMemoryError]) { - System.exit(ExecutorExitCode.OOM) - } else { - System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION) - } - } - } catch { - case oom: OutOfMemoryError => Runtime.getRuntime.halt(ExecutorExitCode.OOM) - case t: Throwable => Runtime.getRuntime.halt(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE) - } - } - } - ) + Thread.setDefaultUncaughtExceptionHandler(ExecutorUncaughtExceptionHandler) } val executorSource = new ExecutorSource(this, executorId) @@ -259,6 +238,11 @@ private[spark] class Executor( } case t: Throwable => { + // Attempt to exit cleanly by informing the driver of our failure. + // If anything goes wrong (or this was a fatal exception), we will delegate to + // the default uncaught exception handler, which will terminate the Executor. + logError("Exception in task ID " + taskId, t) + val serviceTime = System.currentTimeMillis() - taskStart val metrics = attemptedTask.flatMap(t => t.metrics) for (m <- metrics) { @@ -268,10 +252,11 @@ private[spark] class Executor( val reason = ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace, metrics) execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) - // TODO: Should we exit the whole executor here? On the one hand, the failed task may - // have left some weird state around depending on when the exception was thrown, but on - // the other hand, maybe we could detect that when future tasks fail and exit then. - logError("Exception in task ID " + taskId, t) + // Don't forcibly exit unless the exception was inherently fatal, to avoid + // stopping other tasks unnecessarily. + if (Utils.isFatalError(t)) { + ExecutorUncaughtExceptionHandler.uncaughtException(t) + } } } finally { // TODO: Unregister shuffle memory only for ResultTask diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala new file mode 100644 index 0000000000000..b0e984c03964c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import org.apache.spark.Logging +import org.apache.spark.util.Utils + +/** + * The default uncaught exception handler for Executors terminates the whole process, to avoid + * getting into a bad state indefinitely. Since Executors are relatively lightweight, it's better + * to fail fast when things go wrong. + */ +private[spark] object ExecutorUncaughtExceptionHandler + extends Thread.UncaughtExceptionHandler with Logging { + + override def uncaughtException(thread: Thread, exception: Throwable) { + try { + logError("Uncaught exception in thread " + thread, exception) + + // We may have been called from a shutdown hook. If so, we must not call System.exit(). + // (If we do, we will deadlock.) + if (!Utils.inShutdown()) { + if (exception.isInstanceOf[OutOfMemoryError]) { + System.exit(ExecutorExitCode.OOM) + } else { + System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION) + } + } + } catch { + case oom: OutOfMemoryError => Runtime.getRuntime.halt(ExecutorExitCode.OOM) + case t: Throwable => Runtime.getRuntime.halt(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE) + } + } + + def uncaughtException(exception: Throwable) { + uncaughtException(Thread.currentThread(), exception) + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 7968a0691db10..a90b0d475c04e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -206,8 +206,8 @@ private[spark] object EventLoggingListener extends Logging { applicationComplete = filePaths.exists { path => isApplicationCompleteFile(path.getName) } ) } catch { - case t: Throwable => - logError("Exception in parsing logging info from directory %s".format(logDir), t) + case e: Exception => + logError("Exception in parsing logging info from directory %s".format(logDir), e) EventLoggingInfo.empty } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index c9ad2b151daf0..99d305b36a959 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -43,7 +43,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul def enqueueSuccessfulTask( taskSetManager: TaskSetManager, tid: Long, serializedData: ByteBuffer) { getTaskResultExecutor.execute(new Runnable { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { try { val result = serializer.get().deserialize[TaskResult[_]](serializedData) match { case directResult: DirectTaskResult[_] => directResult @@ -70,7 +70,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul case cnf: ClassNotFoundException => val loader = Thread.currentThread.getContextClassLoader taskSetManager.abort("ClassNotFound with classloader: " + loader) - case ex: Throwable => + case ex: Exception => taskSetManager.abort("Exception while deserializing and fetching task: %s".format(ex)) } } @@ -81,7 +81,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul serializedData: ByteBuffer) { var reason : TaskEndReason = UnknownReason getTaskResultExecutor.execute(new Runnable { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { try { if (serializedData != null && serializedData.limit() > 0) { reason = serializer.get().deserialize[TaskEndReason]( @@ -94,7 +94,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul val loader = Utils.getContextOrSparkClassLoader logError( "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) - case ex: Throwable => {} + case ex: Exception => {} } scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index cf6ef0029a861..3a7243a1ba19c 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -148,7 +148,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD private def addShutdownHook() { localDirs.foreach(localDir => Utils.registerShutdownDeleteDir(localDir)) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { logDebug("Shutdown hook called") DiskBlockManager.this.stop() } @@ -162,8 +162,8 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD try { if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) } catch { - case t: Throwable => - logError("Exception while deleting local spark dir: " + localDir, t) + case e: Exception => + logError("Exception while deleting local spark dir: " + localDir, e) } } } diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala index b0b9674856568..a6cbe3aa440ff 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -25,7 +25,6 @@ import tachyon.client.TachyonFile import org.apache.spark.Logging import org.apache.spark.executor.ExecutorExitCode -import org.apache.spark.network.netty.ShuffleSender import org.apache.spark.util.Utils @@ -137,7 +136,7 @@ private[spark] class TachyonBlockManager( private def addShutdownHook() { tachyonDirs.foreach(tachyonDir => Utils.registerShutdownDeleteDir(tachyonDir)) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark tachyon dirs") { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { logDebug("Shutdown hook called") tachyonDirs.foreach { tachyonDir => try { @@ -145,8 +144,8 @@ private[spark] class TachyonBlockManager( Utils.deleteRecursively(tachyonDir, client) } } catch { - case t: Throwable => - logError("Exception while deleting tachyon spark dir: " + tachyonDir, t) + case e: Exception => + logError("Exception while deleting tachyon spark dir: " + tachyonDir, e) } } } diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 8afe09a117ebc..a8d12bb2a0165 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.util import scala.collection.JavaConversions.mapAsJavaMap import scala.concurrent.duration.{Duration, FiniteDuration} -import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} +import akka.actor.{ActorSystem, ExtendedActorSystem} import com.typesafe.config.ConfigFactory import org.apache.log4j.{Level, Logger} @@ -41,7 +41,7 @@ private[spark] object AkkaUtils extends Logging { * If indestructible is set to true, the Actor System will continue running in the event * of a fatal exception. This is used by [[org.apache.spark.executor.Executor]]. */ - def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false, + def createActorSystem(name: String, host: String, port: Int, conf: SparkConf, securityManager: SecurityManager): (ActorSystem, Int) = { val akkaThreads = conf.getInt("spark.akka.threads", 4) @@ -101,12 +101,7 @@ private[spark] object AkkaUtils extends Logging { |akka.log-dead-letters-during-shutdown = $lifecycleEvents """.stripMargin)) - val actorSystem = if (indestructible) { - IndestructibleActorSystem(name, akkaConf) - } else { - ActorSystem(name, akkaConf) - } - + val actorSystem = ActorSystem(name, akkaConf) val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider val boundPort = provider.getDefaultAddress.port.get (actorSystem, boundPort) diff --git a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala deleted file mode 100644 index 4188a869c13da..0000000000000 --- a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -// Must be in akka.actor package as ActorSystemImpl is protected[akka]. -package akka.actor - -import scala.util.control.{ControlThrowable, NonFatal} - -import com.typesafe.config.Config - -/** - * An akka.actor.ActorSystem which refuses to shut down in the event of a fatal exception - * This is necessary as Spark Executors are allowed to recover from fatal exceptions - * (see org.apache.spark.executor.Executor) - */ -object IndestructibleActorSystem { - def apply(name: String, config: Config): ActorSystem = - apply(name, config, ActorSystem.findClassLoader()) - - def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem = - new IndestructibleActorSystemImpl(name, config, classLoader).start() -} - -private[akka] class IndestructibleActorSystemImpl( - override val name: String, - applicationConfig: Config, - classLoader: ClassLoader) - extends ActorSystemImpl(name, applicationConfig, classLoader) { - - protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = { - val fallbackHandler = super.uncaughtExceptionHandler - - new Thread.UncaughtExceptionHandler() { - def uncaughtException(thread: Thread, cause: Throwable): Unit = { - if (isFatalError(cause) && !settings.JvmExitOnFatalError) { - log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + - "ActorSystem [{}] tolerating and continuing.... ", thread.getName, name) - // shutdown() //TODO make it configurable - } else { - fallbackHandler.uncaughtException(thread, cause) - } - } - } - } - - def isFatalError(e: Throwable): Boolean = { - e match { - case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable => - false - case _ => - true - } - } -} diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 95777fbf57d8b..8f7594ada2ba1 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -29,6 +29,7 @@ import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag import scala.util.Try +import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder @@ -41,7 +42,6 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} - /** * Various utility methods used by Spark. */ @@ -1125,4 +1125,28 @@ private[spark] object Utils extends Logging { } } + /** + * Executes the given block, printing and re-throwing any uncaught exceptions. + * This is particularly useful for wrapping code that runs in a thread, to ensure + * that exceptions are printed, and to avoid having to catch Throwable. + */ + def logUncaughtExceptions[T](f: => T): T = { + try { + f + } catch { + case t: Throwable => + logError(s"Uncaught exception in thread ${Thread.currentThread().getName}", t) + throw t + } + } + + /** Returns true if the given exception was fatal. See docs for scala.util.control.NonFatal. */ + def isFatalError(e: Throwable): Boolean = { + e match { + case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable => + false + case _ => + true + } + } } From d9c97ba39723d36d3dcb53a75a3a2565357c0955 Mon Sep 17 00:00:00 2001 From: Bernardo Gomez Palacio Date: Mon, 12 May 2014 11:10:28 -0700 Subject: [PATCH 316/641] SPARK-1806: Upgrade Mesos dependency to 0.18.1 Enabled Mesos (0.18.1) dependency with shaded protobuf Why is this needed? Avoids any protobuf version collision between Mesos and any other dependency in Spark e.g. Hadoop HDFS 2.2+ or 1.0.4. Ticket: https://issues.apache.org/jira/browse/SPARK-1806 * Should close https://issues.apache.org/jira/browse/SPARK-1433 Author berngp Author: Bernardo Gomez Palacio Closes #741 from berngp/feature/SPARK-1806 and squashes the following commits: 5d70646 [Bernardo Gomez Palacio] SPARK-1806: Upgrade Mesos dependency to 0.18.1 --- core/pom.xml | 3 ++- .../apache/spark/executor/MesosExecutorBackend.scala | 2 +- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- pom.xml | 10 +++++++++- project/SparkBuild.scala | 2 +- 5 files changed, 14 insertions(+), 5 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 8fe215ab24289..bab50f5ce2888 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -140,6 +140,7 @@ org.apache.mesos mesos + ${mesos.classifier} io.netty @@ -322,7 +323,7 @@ - + src/main/resources diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 9b56f711e0e0b..74100498bb2bd 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -19,7 +19,7 @@ package org.apache.spark.executor import java.nio.ByteBuffer -import com.google.protobuf.ByteString +import org.apache.mesos.protobuf.ByteString import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver, MesosNativeLibrary} import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index c975f312324ed..2cea1c8cf154a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -24,7 +24,7 @@ import java.util.Collections import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import com.google.protobuf.ByteString +import org.apache.mesos.protobuf.ByteString import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} diff --git a/pom.xml b/pom.xml index c4e1c6be52a1b..dd1d262881b93 100644 --- a/pom.xml +++ b/pom.xml @@ -113,7 +113,8 @@ 2.10.4 2.10 - 0.13.0 + 0.18.1 + shaded-protobuf org.spark-project.akka 2.2.3-shaded-protobuf 1.7.5 @@ -349,6 +350,13 @@ org.apache.mesos mesos ${mesos.version} + ${mesos.classifier} + + + com.google.protobuf + protobuf-java + + commons-net diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 12791e490ae2b..57b3e22f81713 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -343,7 +343,7 @@ object SparkBuild extends Build { "org.spark-project.akka" %% "akka-testkit" % akkaVersion % "test", "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "colt" % "colt" % "1.2.0", - "org.apache.mesos" % "mesos" % "0.13.0", + "org.apache.mesos" % "mesos" % "0.18.1" classifier("shaded-protobuf") exclude("com.google.protobuf", "protobuf-java"), "commons-net" % "commons-net" % "2.2", "net.java.dev.jets3t" % "jets3t" % jets3tVersion excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", From 0e2bde2030f8e455c5a269fc38d4ff05b395ca32 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 12 May 2014 13:05:24 -0700 Subject: [PATCH 317/641] SPARK-1786: Reopening PR 724 Addressing issue in MimaBuild.scala. Author: Ankur Dave Author: Joseph E. Gonzalez Closes #742 from jegonzal/edge_partition_serialization and squashes the following commits: 8ba6e0d [Ankur Dave] Add concatenation operators to MimaBuild.scala cb2ed3a [Joseph E. Gonzalez] addressing missing exclusion in MimaBuild.scala 5d27824 [Ankur Dave] Disable reference tracking to fix serialization test c0a9ae5 [Ankur Dave] Add failing test for EdgePartition Kryo serialization a4a3faa [Joseph E. Gonzalez] Making EdgePartition serializable. --- .../spark/graphx/GraphKryoRegistrator.scala | 9 ++++++--- .../spark/graphx/impl/EdgePartition.scala | 14 +++++++------- .../graphx/impl/EdgePartitionBuilder.scala | 4 ++-- .../graphx/impl/EdgeTripletIterator.scala | 2 +- .../graphx/impl/RoutingTablePartition.scala | 4 ++-- .../graphx/impl/ShippableVertexPartition.scala | 2 +- .../spark/graphx/impl/VertexPartition.scala | 2 +- .../graphx/impl/VertexPartitionBase.scala | 6 +++--- .../graphx/impl/VertexPartitionBaseOps.scala | 4 ++-- ...ala => GraphXPrimitiveKeyOpenHashMap.scala} | 2 +- .../spark/graphx/impl/EdgePartitionSuite.scala | 18 ++++++++++++++++++ project/MimaBuild.scala | 8 +++++--- 12 files changed, 49 insertions(+), 26 deletions(-) rename graphx/src/main/scala/org/apache/spark/graphx/util/collection/{PrimitiveKeyOpenHashMap.scala => GraphXPrimitiveKeyOpenHashMap.scala} (98%) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index d295d0127ac72..f97f329c0e832 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -24,6 +24,9 @@ import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx.impl._ +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.OpenHashSet + /** * Registers GraphX classes with Kryo for improved performance. @@ -43,8 +46,8 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[PartitionStrategy]) kryo.register(classOf[BoundedPriorityQueue[Object]]) kryo.register(classOf[EdgeDirection]) - - // This avoids a large number of hash table lookups. - kryo.setReferences(false) + kryo.register(classOf[GraphXPrimitiveKeyOpenHashMap[VertexId, Int]]) + kryo.register(classOf[OpenHashSet[Int]]) + kryo.register(classOf[OpenHashSet[Long]]) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index 871e81f8d245c..a5c9cd1f8b4e6 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * A collection of edges stored in columnar format, along with any vertex attributes referenced. The @@ -42,12 +42,12 @@ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] class EdgePartition[ @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag, VD: ClassTag]( - @transient val srcIds: Array[VertexId], - @transient val dstIds: Array[VertexId], - @transient val data: Array[ED], - @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int], - @transient val vertices: VertexPartition[VD], - @transient val activeSet: Option[VertexSet] = None + val srcIds: Array[VertexId] = null, + val dstIds: Array[VertexId] = null, + val data: Array[ED] = null, + val index: GraphXPrimitiveKeyOpenHashMap[VertexId, Int] = null, + val vertices: VertexPartition[VD] = null, + val activeSet: Option[VertexSet] = None ) extends Serializable { /** Return a new `EdgePartition` with the specified edge data. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index ecb49bef42e45..4520beb991515 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -23,7 +23,7 @@ import scala.util.Sorting import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: ClassTag]( @@ -41,7 +41,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla val srcIds = new Array[VertexId](edgeArray.size) val dstIds = new Array[VertexId](edgeArray.size) val data = new Array[ED](edgeArray.size) - val index = new PrimitiveKeyOpenHashMap[VertexId, Int] + val index = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int] // Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and // adding them to the index if (edgeArray.length > 0) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index ebb0b9418d65d..56f79a7097fce 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * The Iterator type returned when constructing edge triplets. This could be an anonymous class in diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index 927e32ad0f448..d02e9238adba5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * A message from the edge partition `pid` to the vertex partition containing `vid` specifying that @@ -69,7 +69,7 @@ object RoutingTablePartition { : Iterator[RoutingTableMessage] = { // Determine which positions each vertex id appears in using a map where the low 2 bits // represent src and dst - val map = new PrimitiveKeyOpenHashMap[VertexId, Byte] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, Byte] edgePartition.srcIds.iterator.foreach { srcId => map.changeValue(srcId, 0x1, (b: Byte) => (b | 0x1).toByte) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index f4e221d4e05ae..dca54b8a7da86 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** Stores vertex attributes to ship to an edge partition. */ private[graphx] diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index f1d174720a1ba..55c7a19d1bdab 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] object VertexPartition { /** Construct a `VertexPartition` from the given vertices. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala index 8d9e0204d27f2..34939b24440aa 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] object VertexPartitionBase { /** @@ -32,7 +32,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)]) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map(pair._1) = pair._2 } @@ -45,7 +45,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map.setMerge(pair._1, pair._2, mergeFunc) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index 21ff615feca6c..a4f769b294010 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -25,7 +25,7 @@ import org.apache.spark.Logging import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * An class containing additional operations for subclasses of VertexPartitionBase that provide @@ -224,7 +224,7 @@ private[graphx] abstract class VertexPartitionBaseOps * Construct a new VertexPartition whose index contains only the vertices in the mask. */ def reindex(): Self[VD] = { - val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] + val hashMap = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] val arbitraryMerge = (a: VD, b: VD) => a for ((k, v) <- self.iterator) { hashMap.setMerge(k, v, arbitraryMerge) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala similarity index 98% rename from graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala index 7b02e2ed1a9cb..57b01b6f2e1fb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala @@ -29,7 +29,7 @@ import scala.reflect._ * Under the hood, it uses our OpenHashSet implementation. */ private[graphx] -class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, +class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, @specialized(Long, Int, Double) V: ClassTag]( val keySet: OpenHashSet[K], var _values: Array[V]) extends Iterable[(K, V)] diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index d2e0c01bc35ef..28fd112f2b124 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -22,6 +22,9 @@ import scala.util.Random import org.scalatest.FunSuite +import org.apache.spark.SparkConf +import org.apache.spark.serializer.KryoSerializer + import org.apache.spark.graphx._ class EdgePartitionSuite extends FunSuite { @@ -120,4 +123,19 @@ class EdgePartitionSuite extends FunSuite { assert(!ep.isActive(-1)) assert(ep.numActives == Some(2)) } + + test("Kryo serialization") { + val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) + val a: EdgePartition[Int, Int] = makeEdgePartition(aList) + val conf = new SparkConf() + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + val s = new KryoSerializer(conf).newInstance() + val aSer: EdgePartition[Int, Int] = s.deserialize(s.serialize(a)) + assert(aSer.srcIds.toList === a.srcIds.toList) + assert(aSer.dstIds.toList === a.dstIds.toList) + assert(aSer.data.toList === a.data.toList) + assert(aSer.index != null) + assert(aSer.vertices.iterator.toSet === a.vertices.iterator.toSet) + } } diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index fafc9b36a77d3..e147be7ddaa61 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -30,7 +30,7 @@ object MimaBuild { // Read package-private excludes from file val excludeFilePath = (base.getAbsolutePath + "/.mima-excludes") - val excludeFile = file(excludeFilePath) + val excludeFile = file(excludeFilePath) val packagePrivateList: Seq[String] = if (!excludeFile.exists()) { Seq() @@ -41,10 +41,10 @@ object MimaBuild { // Exclude a single class and its corresponding object def excludeClass(className: String) = { Seq( - excludePackage(className), + excludePackage(className), ProblemFilters.exclude[MissingClassProblem](className), ProblemFilters.exclude[MissingTypesProblem](className), - excludePackage(className + "$"), + excludePackage(className + "$"), ProblemFilters.exclude[MissingClassProblem](className + "$"), ProblemFilters.exclude[MissingTypesProblem](className + "$") ) @@ -78,6 +78,8 @@ object MimaBuild { excludeSparkClass("graphx.VertexRDD") ++ excludeSparkClass("graphx.impl.GraphImpl") ++ excludeSparkClass("graphx.impl.RoutingTable") ++ + excludeSparkClass("graphx.util.collection.PrimitiveKeyOpenHashMap") ++ + excludeSparkClass("graphx.util.collection.GraphXPrimitiveKeyOpenHashMap") ++ excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ excludeSparkClass("mllib.optimization.SquaredGradient") ++ excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ From 1e4a65e69489ff877e6da6f78b1c1306335e373c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 13:20:23 -0700 Subject: [PATCH 318/641] BUILD: Include Hive with default packages when creating a release --- dev/create-release/create-release.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 33552a74920a7..b7a0a01c3bad2 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -109,9 +109,9 @@ make_binary_release() { spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } -make_binary_release "hadoop1" "--hadoop 1.0.4" -make_binary_release "cdh4" "--hadoop 2.0.0-mr1-cdh4.2.0" -make_binary_release "hadoop2" "--with-yarn --hadoop 2.2.0" +make_binary_release "hadoop1" "--with-hive --hadoop 1.0.4" +make_binary_release "cdh4" "--with-hive --hadoop 2.0.0-mr1-cdh4.2.0" +make_binary_release "hadoop2" "--with-hive --with-yarn --hadoop 2.2.0" # Copy data echo "Copying release tarballs" From 7120a2979d0a9f0f54a88b2416be7ca10e74f409 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 12 May 2014 14:16:19 -0700 Subject: [PATCH 319/641] SPARK-1798. Tests should clean up temp files MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Three issues related to temp files that tests generate – these should be touched up for hygiene but are not urgent. Modules have a log4j.properties which directs the unit-test.log output file to a directory like `[module]/target/unit-test.log`. But this ends up creating `[module]/[module]/target/unit-test.log` instead of former. The `work/` directory is not deleted by "mvn clean", in the parent and in modules. Neither is the `checkpoint/` directory created under the various external modules. Many tests create a temp directory, which is not usually deleted. This can be largely resolved by calling `deleteOnExit()` at creation and trying to call `Utils.deleteRecursively` consistently to clean up, sometimes in an `@After` method. _If anyone seconds the motion, I can create a more significant change that introduces a new test trait along the lines of `LocalSparkContext`, which provides management of temp directories for subclasses to take advantage of._ Author: Sean Owen Closes #732 from srowen/SPARK-1798 and squashes the following commits: 5af578e [Sean Owen] Try to consistently delete test temp dirs and files, and set deleteOnExit() for each b21b356 [Sean Owen] Remove work/ and checkpoint/ dirs with mvn clean bdd0f41 [Sean Owen] Remove duplicate module dir in log4j.properties output path for tests --- bagel/src/test/resources/log4j.properties | 2 +- .../scala/org/apache/spark/TestUtils.scala | 1 + .../scala/org/apache/spark/util/Utils.scala | 18 +++--- .../java/org/apache/spark/JavaAPISuite.java | 18 ++---- core/src/test/resources/log4j.properties | 2 +- .../org/apache/spark/CheckpointSuite.scala | 5 +- .../org/apache/spark/FileServerSuite.scala | 18 ++++-- .../scala/org/apache/spark/FileSuite.scala | 56 +++++++++---------- .../WholeTextFileRecordReaderSuite.scala | 4 +- .../spark/rdd/PairRDDFunctionsSuite.scala | 2 + .../scheduler/EventLoggingListenerSuite.scala | 15 +++-- .../spark/scheduler/ReplayListenerSuite.scala | 14 +++-- .../spark/storage/DiskBlockManagerSuite.scala | 30 +++++++--- .../apache/spark/util/FileLoggerSuite.scala | 17 ++++-- .../org/apache/spark/util/UtilsSuite.scala | 1 + .../flume/src/test/resources/log4j.properties | 2 +- .../kafka/src/test/resources/log4j.properties | 2 +- .../mqtt/src/test/resources/log4j.properties | 2 +- .../src/test/resources/log4j.properties | 2 +- .../src/test/resources/log4j.properties | 2 +- .../java/org/apache/spark/Java8APISuite.java | 3 + graphx/src/test/resources/log4j.properties | 2 +- mllib/src/test/resources/log4j.properties | 2 +- .../spark/mllib/util/MLUtilsSuite.scala | 17 ++---- pom.xml | 15 +++++ repl/src/test/resources/log4j.properties | 2 +- .../spark/repl/ExecutorClassLoaderSuite.scala | 24 ++++++-- .../org/apache/spark/repl/ReplSuite.scala | 3 + .../apache/spark/sql/InsertIntoSuite.scala | 12 ++++ .../org/apache/spark/sql/hive/TestHive.scala | 1 + .../streaming/util/MasterFailureTest.scala | 3 + .../apache/spark/streaming/JavaAPISuite.java | 5 +- streaming/src/test/resources/log4j.properties | 2 +- .../spark/streaming/CheckpointSuite.scala | 2 + .../spark/streaming/InputStreamsSuite.scala | 1 + 35 files changed, 193 insertions(+), 114 deletions(-) diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties index 5cdcf35b23a6c..30b4baa4d714a 100644 --- a/bagel/src/test/resources/log4j.properties +++ b/bagel/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=bagel/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 8ae02154823ee..885c6829a2d72 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -43,6 +43,7 @@ private[spark] object TestUtils { */ def createJarWithClasses(classNames: Seq[String], value: String = ""): URL = { val tempDir = Files.createTempDir() + tempDir.deleteOnExit() val files = for (name <- classNames) yield createCompiledClass(name, tempDir, value) val jarFile = new File(tempDir, "testJar-%s.jar".format(System.currentTimeMillis())) createJar(files, jarFile) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 8f7594ada2ba1..0631e54237923 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -586,15 +586,17 @@ private[spark] object Utils extends Logging { * Don't follow directories if they are symlinks. */ def deleteRecursively(file: File) { - if ((file.isDirectory) && !isSymlink(file)) { - for (child <- listFilesSafely(file)) { - deleteRecursively(child) + if (file != null) { + if ((file.isDirectory) && !isSymlink(file)) { + for (child <- listFilesSafely(file)) { + deleteRecursively(child) + } } - } - if (!file.delete()) { - // Delete can also fail if the file simply did not exist - if (file.exists()) { - throw new IOException("Failed to delete: " + file.getAbsolutePath) + if (!file.delete()) { + // Delete can also fail if the file simply did not exist + if (file.exists()) { + throw new IOException("Failed to delete: " + file.getAbsolutePath) + } } } } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 1912015827927..3dd79243ab5bd 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -18,7 +18,6 @@ package org.apache.spark; import java.io.*; -import java.lang.StringBuilder; import java.util.*; import scala.Tuple2; @@ -49,16 +48,20 @@ import org.apache.spark.partial.PartialResult; import org.apache.spark.storage.StorageLevel; import org.apache.spark.util.StatCounter; +import org.apache.spark.util.Utils; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; // see http://stackoverflow.com/questions/758570/. public class JavaAPISuite implements Serializable { private transient JavaSparkContext sc; + private transient File tempDir; @Before public void setUp() { sc = new JavaSparkContext("local", "JavaAPISuite"); + tempDir = Files.createTempDir(); + tempDir.deleteOnExit(); } @After @@ -67,6 +70,7 @@ public void tearDown() { sc = null; // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port"); + Utils.deleteRecursively(tempDir); } static class ReverseIntComparator implements Comparator, Serializable { @@ -611,7 +615,6 @@ public void glom() { @Test public void textFiles() throws IOException { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); rdd.saveAsTextFile(outputDir); @@ -630,7 +633,6 @@ public void wholeTextFiles() throws IOException { byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); byte[] content2 = "spark is also easy to use.\n".getBytes("utf-8"); - File tempDir = Files.createTempDir(); String tempDirName = tempDir.getAbsolutePath(); DataOutputStream ds = new DataOutputStream(new FileOutputStream(tempDirName + "/part-00000")); ds.write(content1); @@ -653,7 +655,6 @@ public void wholeTextFiles() throws IOException { @Test public void textFilesCompressed() throws IOException { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); rdd.saveAsTextFile(outputDir, DefaultCodec.class); @@ -667,7 +668,6 @@ public void textFilesCompressed() throws IOException { @SuppressWarnings("unchecked") @Test public void sequenceFile() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -697,7 +697,6 @@ public Tuple2 call(Tuple2 pair) { @SuppressWarnings("unchecked") @Test public void writeWithNewAPIHadoopFile() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -728,7 +727,6 @@ public String call(Tuple2 x) { @SuppressWarnings("unchecked") @Test public void readWithNewAPIHadoopFile() throws IOException { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -758,7 +756,6 @@ public String call(Tuple2 x) { @Test public void objectFilesOfInts() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); rdd.saveAsObjectFile(outputDir); @@ -771,7 +768,6 @@ public void objectFilesOfInts() { @SuppressWarnings("unchecked") @Test public void objectFilesOfComplexTypes() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -788,7 +784,6 @@ public void objectFilesOfComplexTypes() { @SuppressWarnings("unchecked") @Test public void hadoopFile() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -818,7 +813,6 @@ public String call(Tuple2 x) { @SuppressWarnings("unchecked") @Test public void hadoopFileCompressed() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output_compressed").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -948,7 +942,6 @@ public String call(Integer t) throws Exception { @Test public void checkpointAndComputation() { - File tempDir = Files.createTempDir(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); sc.setCheckpointDir(tempDir.getAbsolutePath()); Assert.assertEquals(false, rdd.isCheckpointed()); @@ -960,7 +953,6 @@ public void checkpointAndComputation() { @Test public void checkpointAndRestore() { - File tempDir = Files.createTempDir(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); sc.setCheckpointDir(tempDir.getAbsolutePath()); Assert.assertEquals(false, rdd.isCheckpointed()); diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index f6fef03689a7c..26b73a1b39744 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=core/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index d2555b7c052c1..64933f4b1046d 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -35,6 +35,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { override def beforeEach() { super.beforeEach() checkpointDir = File.createTempFile("temp", "") + checkpointDir.deleteOnExit() checkpointDir.delete() sc = new SparkContext("local", "test") sc.setCheckpointDir(checkpointDir.toString) @@ -42,9 +43,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { override def afterEach() { super.afterEach() - if (checkpointDir != null) { - checkpointDir.delete() - } + Utils.deleteRecursively(checkpointDir) } test("basic checkpointing") { diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index d651fbbac4e97..7e18f45de7b5b 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -24,9 +24,11 @@ import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.spark.SparkContext._ +import org.apache.spark.util.Utils class FileServerSuite extends FunSuite with LocalSparkContext { + @transient var tmpDir: File = _ @transient var tmpFile: File = _ @transient var tmpJarUrl: String = _ @@ -38,15 +40,18 @@ class FileServerSuite extends FunSuite with LocalSparkContext { override def beforeAll() { super.beforeAll() - val tmpDir = new File(Files.createTempDir(), "test") - tmpDir.mkdir() - val textFile = new File(tmpDir, "FileServerSuite.txt") + tmpDir = Files.createTempDir() + tmpDir.deleteOnExit() + val testTempDir = new File(tmpDir, "test") + testTempDir.mkdir() + + val textFile = new File(testTempDir, "FileServerSuite.txt") val pw = new PrintWriter(textFile) pw.println("100") pw.close() - val jarFile = new File(tmpDir, "test.jar") + val jarFile = new File(testTempDir, "test.jar") val jarStream = new FileOutputStream(jarFile) val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest()) System.setProperty("spark.authenticate", "false") @@ -70,6 +75,11 @@ class FileServerSuite extends FunSuite with LocalSparkContext { tmpJarUrl = jarFile.toURI.toURL.toString } + override def afterAll() { + super.afterAll() + Utils.deleteRecursively(tmpDir) + } + test("Distributing files locally") { sc = new SparkContext("local[4]", "test") sc.addFile(tmpFile.toString) diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index b9b668d3cc62a..1f2206b1f0379 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -30,12 +30,24 @@ import org.apache.hadoop.mapreduce.Job import org.scalatest.FunSuite import org.apache.spark.SparkContext._ +import org.apache.spark.util.Utils class FileSuite extends FunSuite with LocalSparkContext { + var tempDir: File = _ + + override def beforeEach() { + super.beforeEach() + tempDir = Files.createTempDir() + tempDir.deleteOnExit() + } + + override def afterEach() { + super.afterEach() + Utils.deleteRecursively(tempDir) + } test("text files") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 4) nums.saveAsTextFile(outputDir) @@ -49,7 +61,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("text files (compressed)") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val normalDir = new File(tempDir, "output_normal").getAbsolutePath val compressedOutputDir = new File(tempDir, "output_compressed").getAbsolutePath val codec = new DefaultCodec() @@ -71,7 +82,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFiles") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (x, "a" * x)) // (1,a), (2,aa), (3,aaa) nums.saveAsSequenceFile(outputDir) @@ -82,7 +92,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFile (compressed)") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val normalDir = new File(tempDir, "output_normal").getAbsolutePath val compressedOutputDir = new File(tempDir, "output_compressed").getAbsolutePath val codec = new DefaultCodec() @@ -104,7 +113,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFile with writable key") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), "a" * x)) nums.saveAsSequenceFile(outputDir) @@ -115,7 +123,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFile with writable value") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (x, new Text("a" * x))) nums.saveAsSequenceFile(outputDir) @@ -126,7 +133,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFile with writable key and value") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) nums.saveAsSequenceFile(outputDir) @@ -137,7 +143,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("implicit conversions in reading SequenceFiles") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (x, "a" * x)) // (1,a), (2,aa), (3,aaa) nums.saveAsSequenceFile(outputDir) @@ -154,7 +159,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("object files of ints") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 4) nums.saveAsObjectFile(outputDir) @@ -165,7 +169,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("object files of complex types") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (x, "a" * x)) nums.saveAsObjectFile(outputDir) @@ -177,7 +180,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("write SequenceFile using new Hadoop API") { import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) nums.saveAsNewAPIHadoopFile[SequenceFileOutputFormat[IntWritable, Text]]( @@ -189,7 +191,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("read SequenceFile using new Hadoop API") { import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) nums.saveAsSequenceFile(outputDir) @@ -200,7 +201,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("file caching") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val out = new FileWriter(tempDir + "/input") out.write("Hello world!\n") out.write("What's up?\n") @@ -214,67 +214,61 @@ class FileSuite extends FunSuite with LocalSparkContext { test ("prevent user from overwriting the empty directory (old Hadoop API)") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1) intercept[FileAlreadyExistsException] { - randomRDD.saveAsTextFile(tempdir.getPath) + randomRDD.saveAsTextFile(tempDir.getPath) } } test ("prevent user from overwriting the non-empty directory (old Hadoop API)") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1) - randomRDD.saveAsTextFile(tempdir.getPath + "/output") - assert(new File(tempdir.getPath + "/output/part-00000").exists() === true) + randomRDD.saveAsTextFile(tempDir.getPath + "/output") + assert(new File(tempDir.getPath + "/output/part-00000").exists() === true) intercept[FileAlreadyExistsException] { - randomRDD.saveAsTextFile(tempdir.getPath + "/output") + randomRDD.saveAsTextFile(tempDir.getPath + "/output") } } test ("prevent user from overwriting the empty directory (new Hadoop API)") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) intercept[FileAlreadyExistsException] { - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempdir.getPath) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath) } } test ("prevent user from overwriting the non-empty directory (new Hadoop API)") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempdir.getPath + "/output") - assert(new File(tempdir.getPath + "/output/part-r-00000").exists() === true) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath + "/output") + assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) intercept[FileAlreadyExistsException] { - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempdir.getPath) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath) } } test ("save Hadoop Dataset through old Hadoop API") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) val job = new JobConf() job.setOutputKeyClass(classOf[String]) job.setOutputValueClass(classOf[String]) job.set("mapred.output.format.class", classOf[TextOutputFormat[String, String]].getName) - job.set("mapred.output.dir", tempdir.getPath + "/outputDataset_old") + job.set("mapred.output.dir", tempDir.getPath + "/outputDataset_old") randomRDD.saveAsHadoopDataset(job) - assert(new File(tempdir.getPath + "/outputDataset_old/part-00000").exists() === true) + assert(new File(tempDir.getPath + "/outputDataset_old/part-00000").exists() === true) } test ("save Hadoop Dataset through new Hadoop API") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) val job = new Job(sc.hadoopConfiguration) job.setOutputKeyClass(classOf[String]) job.setOutputValueClass(classOf[String]) job.setOutputFormatClass(classOf[NewTextOutputFormat[String, String]]) - job.getConfiguration.set("mapred.output.dir", tempdir.getPath + "/outputDataset_new") + job.getConfiguration.set("mapred.output.dir", tempDir.getPath + "/outputDataset_new") randomRDD.saveAsNewAPIHadoopDataset(job.getConfiguration) - assert(new File(tempdir.getPath + "/outputDataset_new/part-r-00000").exists() === true) + assert(new File(tempDir.getPath + "/outputDataset_new/part-r-00000").exists() === true) } } diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index 33d6de9a76405..d5ebfb3f3fae1 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -31,6 +31,7 @@ import org.scalatest.FunSuite import org.apache.hadoop.io.Text import org.apache.spark.SparkContext +import org.apache.spark.util.Utils /** * Tests the correctness of @@ -67,6 +68,7 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { test("Correctness of WholeTextFileRecordReader.") { val dir = Files.createTempDir() + dir.deleteOnExit() println(s"Local disk address is ${dir.toString}.") WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => @@ -86,7 +88,7 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { s"file $filename contents can not match.") } - dir.delete() + Utils.deleteRecursively(dir) } } diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 8f3e6bd21b752..1230565ea5b7e 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -236,11 +236,13 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { test("zero-partition RDD") { val emptyDir = Files.createTempDir() + emptyDir.deleteOnExit() val file = sc.textFile(emptyDir.getAbsolutePath) assert(file.partitions.size == 0) assert(file.collect().toList === Nil) // Test that a shuffle on the file works, because this used to be a bug assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) + emptyDir.delete() } test("keys and values") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 95f5bcd855665..21e3db34b8b7a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.scheduler import scala.collection.mutable import scala.io.Source -import scala.util.Try import com.google.common.io.Files import org.apache.hadoop.fs.{FileStatus, Path} @@ -30,6 +29,8 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{JsonProtocol, Utils} +import java.io.File + /** * Test whether EventLoggingListener logs events properly. * @@ -43,11 +44,17 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { "org.apache.spark.io.LZFCompressionCodec", "org.apache.spark.io.SnappyCompressionCodec" ) - private val testDir = Files.createTempDir() - private val logDirPath = Utils.getFilePath(testDir, "spark-events") + private var testDir: File = _ + private var logDirPath: Path = _ + + before { + testDir = Files.createTempDir() + testDir.deleteOnExit() + logDirPath = Utils.getFilePath(testDir, "spark-events") + } after { - Try { fileSystem.delete(logDirPath, true) } + Utils.deleteRecursively(testDir) } test("Parse names of special files") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index d1fe1fc348961..d81499ac6abef 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -17,9 +17,7 @@ package org.apache.spark.scheduler -import java.io.PrintWriter - -import scala.util.Try +import java.io.{File, PrintWriter} import com.google.common.io.Files import org.json4s.jackson.JsonMethods._ @@ -39,11 +37,15 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { "org.apache.spark.io.LZFCompressionCodec", "org.apache.spark.io.SnappyCompressionCodec" ) - private val testDir = Files.createTempDir() + private var testDir: File = _ + + before { + testDir = Files.createTempDir() + testDir.deleteOnExit() + } after { - Try { fileSystem.delete(Utils.getFilePath(testDir, "events.txt"), true) } - Try { fileSystem.delete(Utils.getFilePath(testDir, "test-replay"), true) } + Utils.deleteRecursively(testDir) } test("Simple replay") { diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 42bfbf1bdfc74..2167718fd2428 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -23,18 +23,16 @@ import scala.collection.mutable import scala.language.reflectiveCalls import com.google.common.io.Files -import org.scalatest.{BeforeAndAfterEach, FunSuite} +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.apache.spark.SparkConf +import org.apache.spark.util.Utils -class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { +class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll { private val testConf = new SparkConf(false) - val rootDir0 = Files.createTempDir() - rootDir0.deleteOnExit() - val rootDir1 = Files.createTempDir() - rootDir1.deleteOnExit() - val rootDirs = rootDir0.getName + "," + rootDir1.getName - println("Created root dirs: " + rootDirs) + private var rootDir0: File = _ + private var rootDir1: File = _ + private var rootDirs: String = _ // This suite focuses primarily on consolidation features, // so we coerce consolidation if not already enabled. @@ -48,6 +46,22 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { var diskBlockManager: DiskBlockManager = _ + override def beforeAll() { + super.beforeAll() + rootDir0 = Files.createTempDir() + rootDir0.deleteOnExit() + rootDir1 = Files.createTempDir() + rootDir1.deleteOnExit() + rootDirs = rootDir0.getName + "," + rootDir1.getName + println("Created root dirs: " + rootDirs) + } + + override def afterAll() { + super.afterAll() + Utils.deleteRecursively(rootDir0) + Utils.deleteRecursively(rootDir1) + } + override def beforeEach() { diskBlockManager = new DiskBlockManager(shuffleBlockManager, rootDirs) shuffleBlockManager.idToSegmentMap.clear() diff --git a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala index f675e1e5b4981..44332fc8dbc23 100644 --- a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala @@ -17,10 +17,9 @@ package org.apache.spark.util -import java.io.IOException +import java.io.{File, IOException} import scala.io.Source -import scala.util.Try import com.google.common.io.Files import org.apache.hadoop.fs.Path @@ -38,12 +37,18 @@ class FileLoggerSuite extends FunSuite with BeforeAndAfter { "org.apache.spark.io.LZFCompressionCodec", "org.apache.spark.io.SnappyCompressionCodec" ) - private val testDir = Files.createTempDir() - private val logDirPath = Utils.getFilePath(testDir, "test-file-logger") - private val logDirPathString = logDirPath.toString + private var testDir: File = _ + private var logDirPath: Path = _ + private var logDirPathString: String = _ + + before { + testDir = Files.createTempDir() + logDirPath = Utils.getFilePath(testDir, "test-file-logger") + logDirPathString = logDirPath.toString + } after { - Try { fileSystem.delete(logDirPath, true) } + Utils.deleteRecursively(testDir) } test("Simple logging") { diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index eb7fb6318262b..cf9e20d347ddd 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -112,6 +112,7 @@ class UtilsSuite extends FunSuite { test("reading offset bytes of a file") { val tmpDir2 = Files.createTempDir() + tmpDir2.deleteOnExit() val f1Path = tmpDir2 + "/f1" val f1 = new FileOutputStream(f1Path) f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties index d1bd73a8430e1..45d2ec676df66 100644 --- a/external/flume/src/test/resources/log4j.properties +++ b/external/flume/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=external/flume/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka/src/test/resources/log4j.properties index 38910d113050a..45d2ec676df66 100644 --- a/external/kafka/src/test/resources/log4j.properties +++ b/external/kafka/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=external/kafka/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties index d0462c7336df5..45d2ec676df66 100644 --- a/external/mqtt/src/test/resources/log4j.properties +++ b/external/mqtt/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=external/mqtt/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties index c918335fcdc70..45d2ec676df66 100644 --- a/external/twitter/src/test/resources/log4j.properties +++ b/external/twitter/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=external/twitter/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties index 304683dd0bac3..45d2ec676df66 100644 --- a/external/zeromq/src/test/resources/log4j.properties +++ b/external/zeromq/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=external/zeromq/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java index feabca6733484..84d3b6f243c72 100644 --- a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java +++ b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java @@ -39,6 +39,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.*; +import org.apache.spark.util.Utils; /** * Most of these tests replicate org.apache.spark.JavaAPISuite using java 8 @@ -249,6 +250,7 @@ public void mapPartitions() { @Test public void sequenceFile() { File tempDir = Files.createTempDir(); + tempDir.deleteOnExit(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -265,6 +267,7 @@ public void sequenceFile() { JavaPairRDD readRDD = sc.sequenceFile(outputDir, IntWritable.class, Text.class) .mapToPair(pair -> new Tuple2(pair._1().get(), pair._2().toString())); Assert.assertEquals(pairs, readRDD.collect()); + Utils.deleteRecursively(tempDir); } @Test diff --git a/graphx/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties index 85e57f0c4b504..26b73a1b39744 100644 --- a/graphx/src/test/resources/log4j.properties +++ b/graphx/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=graphx/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties index 4265ba6e5de33..ddfc4ac6b23ed 100644 --- a/mllib/src/test/resources/log4j.properties +++ b/mllib/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=mllib/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 3f64baf6fe41f..3d05fb68988c8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -32,6 +32,7 @@ import com.google.common.io.Files import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils._ +import org.apache.spark.util.Utils class MLUtilsSuite extends FunSuite with LocalSparkContext { @@ -67,6 +68,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { |-1 2:4.0 4:5.0 6:6.0 """.stripMargin val tempDir = Files.createTempDir() + tempDir.deleteOnExit() val file = new File(tempDir.getPath, "part-00000") Files.write(lines, file, Charsets.US_ASCII) val path = tempDir.toURI.toString @@ -90,7 +92,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { assert(multiclassPoints(1).label === -1.0) assert(multiclassPoints(2).label === -1.0) - deleteQuietly(tempDir) + Utils.deleteRecursively(tempDir) } test("saveAsLibSVMFile") { @@ -107,7 +109,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { .toSet val expected = Set("1.1 1:1.23 3:4.56", "0.0 1:1.01 2:2.02 3:3.03") assert(lines === expected) - deleteQuietly(tempDir) + Utils.deleteRecursively(tempDir) } test("appendBias") { @@ -158,16 +160,5 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { } } - /** Delete a file/directory quietly. */ - def deleteQuietly(f: File) { - if (f.isDirectory) { - f.listFiles().foreach(deleteQuietly) - } - try { - f.delete() - } catch { - case _: Throwable => - } - } } diff --git a/pom.xml b/pom.xml index dd1d262881b93..5542a32a91368 100644 --- a/pom.xml +++ b/pom.xml @@ -796,6 +796,21 @@ + + org.apache.maven.plugins + maven-clean-plugin + 2.5 + + + + work + + + checkpoint + + + + diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties index a6d33e69d21f7..9c4896e49698c 100644 --- a/repl/src/test/resources/log4j.properties +++ b/repl/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=repl/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala index 336df988a1b7f..c0af7ceb6d3ef 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.repl import java.io.File -import java.net.URLClassLoader +import java.net.{URL, URLClassLoader} import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite @@ -26,21 +26,35 @@ import org.scalatest.FunSuite import com.google.common.io.Files import org.apache.spark.TestUtils +import org.apache.spark.util.Utils class ExecutorClassLoaderSuite extends FunSuite with BeforeAndAfterAll { val childClassNames = List("ReplFakeClass1", "ReplFakeClass2") val parentClassNames = List("ReplFakeClass1", "ReplFakeClass2", "ReplFakeClass3") - val tempDir1 = Files.createTempDir() - val tempDir2 = Files.createTempDir() - val url1 = "file://" + tempDir1 - val urls2 = List(tempDir2.toURI.toURL).toArray + var tempDir1: File = _ + var tempDir2: File = _ + var url1: String = _ + var urls2: Array[URL] = _ override def beforeAll() { + super.beforeAll() + tempDir1 = Files.createTempDir() + tempDir1.deleteOnExit() + tempDir2 = Files.createTempDir() + tempDir2.deleteOnExit() + url1 = "file://" + tempDir1 + urls2 = List(tempDir2.toURI.toURL).toArray childClassNames.foreach(TestUtils.createCompiledClass(_, tempDir1, "1")) parentClassNames.foreach(TestUtils.createCompiledClass(_, tempDir2, "2")) } + override def afterAll() { + super.afterAll() + Utils.deleteRecursively(tempDir1) + Utils.deleteRecursively(tempDir2) + } + test("child first") { val parentLoader = new URLClassLoader(urls2, null) val classLoader = new ExecutorClassLoader(url1, parentLoader, true) diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 566d96e16ed83..95460aa205331 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -26,6 +26,7 @@ import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.commons.lang3.StringEscapeUtils +import org.apache.spark.util.Utils class ReplSuite extends FunSuite { @@ -178,6 +179,7 @@ class ReplSuite extends FunSuite { test("interacting with files") { val tempDir = Files.createTempDir() + tempDir.deleteOnExit() val out = new FileWriter(tempDir + "/input") out.write("Hello world!\n") out.write("What's up?\n") @@ -196,6 +198,7 @@ class ReplSuite extends FunSuite { assertContains("res0: Long = 3", output) assertContains("res1: Long = 3", output) assertContains("res2: Long = 3", output) + Utils.deleteRecursively(tempDir) } test("local-cluster mode") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala index 73d87963b3a0d..4f0b85f26254b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala @@ -29,6 +29,7 @@ class InsertIntoSuite extends QueryTest { test("insertInto() created parquet file") { val testFilePath = File.createTempFile("sparkSql", "pqt") testFilePath.delete() + testFilePath.deleteOnExit() val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) testFile.registerAsTable("createAndInsertTest") @@ -76,11 +77,14 @@ class InsertIntoSuite extends QueryTest { sql("SELECT * FROM createAndInsertTest"), testData.collect().toSeq ) + + testFilePath.delete() } test("INSERT INTO parquet table") { val testFilePath = File.createTempFile("sparkSql", "pqt") testFilePath.delete() + testFilePath.deleteOnExit() val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) testFile.registerAsTable("createAndInsertSQLTest") @@ -126,23 +130,31 @@ class InsertIntoSuite extends QueryTest { sql("SELECT * FROM createAndInsertSQLTest"), testData.collect().toSeq ) + + testFilePath.delete() } test("Double create fails when allowExisting = false") { val testFilePath = File.createTempFile("sparkSql", "pqt") testFilePath.delete() + testFilePath.deleteOnExit() val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) intercept[RuntimeException] { createParquetFile[TestData](testFilePath.getCanonicalPath, allowExisting = false) } + + testFilePath.delete() } test("Double create does not fail when allowExisting = true") { val testFilePath = File.createTempFile("sparkSql", "pqt") testFilePath.delete() + testFilePath.deleteOnExit() val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) createParquetFile[TestData](testFilePath.getCanonicalPath, allowExisting = true) + + testFilePath.delete() } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 3ad66a3d7f45f..fa7d010459c63 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -99,6 +99,7 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { val hiveFilesTemp = File.createTempFile("catalystHiveFiles", "") hiveFilesTemp.delete() hiveFilesTemp.mkdir() + hiveFilesTemp.deleteOnExit() val inRepoTests = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala index c48a38590e060..b3ed302db6a38 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala @@ -21,6 +21,7 @@ import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.streaming._ import org.apache.spark.streaming.dstream.{DStream, ForEachDStream} +import org.apache.spark.util.Utils import StreamingContext._ import scala.util.Random @@ -380,6 +381,7 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) override def run() { val localTestDir = Files.createTempDir() + localTestDir.deleteOnExit() var fs = testDir.getFileSystem(new Configuration()) val maxTries = 3 try { @@ -421,6 +423,7 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) case e: Exception => logWarning("File generating in killing thread", e) } finally { fs.close() + Utils.deleteRecursively(localTestDir) } } } diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index f9bfb9b7444cc..ce58cb12a4564 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -17,7 +17,6 @@ package org.apache.spark.streaming; -import org.apache.spark.streaming.api.java.*; import scala.Tuple2; import org.junit.Assert; @@ -37,6 +36,8 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.*; import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.api.java.*; +import org.apache.spark.util.Utils; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; @@ -1606,6 +1607,7 @@ public void testCheckpointMasterRecovery() throws InterruptedException { Arrays.asList(8,7)); File tempDir = Files.createTempDir(); + tempDir.deleteOnExit(); ssc.checkpoint(tempDir.getAbsolutePath()); JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1); @@ -1627,6 +1629,7 @@ public Integer call(String s) throws Exception { // will be re-processed after recovery List> finalResult = JavaCheckpointTestUtils.runStreams(ssc, 2, 3); assertOrderInvariantEquals(expectedFinal, finalResult.subList(1, 3)); + Utils.deleteRecursively(tempDir); } diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index 063529a9cbc67..45d2ec676df66 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=streaming/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 25739956cb889..d20a7b728c741 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -232,6 +232,7 @@ class CheckpointSuite extends TestSuiteBase { test("recovery with file input stream") { // Set up the streaming context and input streams val testDir = Files.createTempDir() + testDir.deleteOnExit() var ssc = new StreamingContext(master, framework, Seconds(1)) ssc.checkpoint(checkpointDir) val fileStream = ssc.textFileStream(testDir.toString) @@ -326,6 +327,7 @@ class CheckpointSuite extends TestSuiteBase { ) // To ensure that all the inputs were received correctly assert(expectedOutput.last === output.last) + Utils.deleteRecursively(testDir) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 3fa254065cc44..cd0aa4d0dce70 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -98,6 +98,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Set up the streaming context and input streams val testDir = Files.createTempDir() + testDir.deleteOnExit() val ssc = new StreamingContext(conf, batchDuration) val fileStream = ssc.textFileStream(testDir.toString) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] From 8586bf564fe010dfc19ef26874472a6f85e355fb Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 12 May 2014 14:17:25 -0700 Subject: [PATCH 320/641] SPARK-1802. Audit dependency graph when Spark is built with -Phive This initial commit resolves the conflicts in the Hive profiles as noted in https://issues.apache.org/jira/browse/SPARK-1802 . Most of the fix was to note that Hive drags in Avro, and so if the hive module depends on Spark's version of the `avro-*` dependencies, it will pull in our exclusions as needed too. But I found we need to copy some exclusions between the two Avro dependencies to get this right. And then had to squash some commons-logging intrusions. This turned up another annoying find, that `hive-exec` is basically an "assembly" artifact that _also_ packages all of its transitive dependencies. This means the final assembly shows lots of collisions between itself and its dependencies, and even other project dependencies. I have a TODO to examine whether that is going to be a deal-breaker or not. In the meantime I'm going to tack on a second commit to this PR that will also fix some similar, last collisions in the YARN profile. Author: Sean Owen Closes #744 from srowen/SPARK-1802 and squashes the following commits: a856604 [Sean Owen] Resolve JAR version conflicts specific to Hive profile --- pom.xml | 16 ++++++++++++++++ sql/hive/pom.xml | 21 +++++++++++++++++++++ 2 files changed, 37 insertions(+) diff --git a/pom.xml b/pom.xml index 5542a32a91368..3554efafb976a 100644 --- a/pom.xml +++ b/pom.xml @@ -535,6 +535,22 @@ io.netty netty + + org.mortbay.jetty + jetty + + + org.mortbay.jetty + jetty-util + + + org.mortbay.jetty + servlet-api + + + org.apache.velocity + velocity + diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 889d249146b8c..8b32451d76045 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -52,6 +52,12 @@ org.apache.hive hive-exec ${hive.version} + + + commons-logging + commons-logging + + org.codehaus.jackson @@ -61,6 +67,21 @@ org.apache.hive hive-serde ${hive.version} + + + commons-logging + commons-logging + + + commons-logging + commons-logging-api + + + + + + org.apache.avro + avro org.scalatest From 3ce526b168050c572a1feee8e0121e1426f7d9ee Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 17:09:13 -0700 Subject: [PATCH 321/641] Rename testExecutorEnvs --> executorEnvs. This was changed, but in fact, it's used for things other than tests. So I've changed it back. Author: Patrick Wendell Closes #747 from pwendell/executor-env and squashes the following commits: 36a60a5 [Patrick Wendell] Rename testExecutorEnvs --> executorEnvs. --- .../main/scala/org/apache/spark/SparkContext.scala | 11 +++++------ .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 2 +- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- 4 files changed, 8 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e6121a705497c..35beff0f8ef3d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -276,19 +276,18 @@ class SparkContext(config: SparkConf) extends Logging { .getOrElse(512) // Environment variables to pass to our executors. - // NOTE: This should only be used for test related settings. - private[spark] val testExecutorEnvs = HashMap[String, String]() + private[spark] val executorEnvs = HashMap[String, String]() // Convert java options to env vars as a work around // since we can't set env vars directly in sbt. for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { - testExecutorEnvs(envKey) = value + executorEnvs(envKey) = value } // The Mesos scheduler backend relies on this environment variable to set executor memory. // TODO: Set this only in the Mesos scheduler. - testExecutorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" - testExecutorEnvs ++= conf.getExecutorEnv + executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" + executorEnvs ++= conf.getExecutorEnv // Set SPARK_USER for user who is running SparkContext. val sparkUser = Option { @@ -296,7 +295,7 @@ class SparkContext(config: SparkConf) extends Logging { }.getOrElse { SparkContext.SPARK_UNKNOWN_USER } - testExecutorEnvs("SPARK_USER") = sparkUser + executorEnvs("SPARK_USER") = sparkUser // Create and start the scheduler private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index cefa41729964a..933f6e0571518 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -54,7 +54,7 @@ private[spark] class SparkDeploySchedulerBackend( } val command = Command( - "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.testExecutorEnvs, + "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, classPathEntries, libraryPathEntries, extraJavaOpts) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 2cd9d6c12eaf7..cbe9bb093d1c9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -122,7 +122,7 @@ private[spark] class CoarseMesosSchedulerBackend( val extraLibraryPath = conf.getOption(libraryPathOption).map(p => s"-Djava.library.path=$p") val extraOpts = Seq(extraJavaOpts, extraLibraryPath).flatten.mkString(" ") - sc.testExecutorEnvs.foreach { case (key, value) => + sc.executorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() .setName(key) .setValue(value) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 2cea1c8cf154a..ff8356f67681e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -90,7 +90,7 @@ private[spark] class MesosSchedulerBackend( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) val environment = Environment.newBuilder() - sc.testExecutorEnvs.foreach { case (key, value) => + sc.executorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() .setName(key) .setValue(value) From 925d8b249b84d2706c52f0d1e29fb8dcd6de452e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 17:27:28 -0700 Subject: [PATCH 322/641] SPARK-1623: Use File objects instead of String's in HTTPBroadcast This seems strictly better, and I think it's justified only the grounds of clean-up. It might also fix issues with path conversions, but I haven't yet isolated any instance of that happening. /cc @srowen @tdas Author: Patrick Wendell Closes #749 from pwendell/broadcast-cleanup and squashes the following commits: d6d54f2 [Patrick Wendell] SPARK-1623: Use File objects instead of string's in HTTPBroadcast --- .../scala/org/apache/spark/broadcast/HttpBroadcast.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 78fc286e5192c..4f6cabaff2b99 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -112,7 +112,7 @@ private[spark] object HttpBroadcast extends Logging { private var securityManager: SecurityManager = null // TODO: This shouldn't be a global variable so that multiple SparkContexts can coexist - private val files = new TimeStampedHashSet[String] + private val files = new TimeStampedHashSet[File] private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES).toInt private var compressionCodec: CompressionCodec = null private var cleaner: MetadataCleaner = null @@ -173,7 +173,7 @@ private[spark] object HttpBroadcast extends Logging { val serOut = ser.serializeStream(out) serOut.writeObject(value) serOut.close() - files += file.getAbsolutePath + files += file } def read[T: ClassTag](id: Long): T = { @@ -216,7 +216,7 @@ private[spark] object HttpBroadcast extends Logging { SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) if (removeFromDriver) { val file = getFile(id) - files.remove(file.toString) + files.remove(file) deleteBroadcastFile(file) } } @@ -232,7 +232,7 @@ private[spark] object HttpBroadcast extends Logging { val (file, time) = (entry.getKey, entry.getValue) if (time < cleanupTime) { iterator.remove() - deleteBroadcastFile(new File(file.toString)) + deleteBroadcastFile(file) } } } From 4b31f4ec7efab8eabf956284a99bfd96a58b79f7 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 12 May 2014 17:35:29 -0700 Subject: [PATCH 323/641] SPARK-1802. (Addendium) Audit dependency graph when Spark is built with -Pyarn Following on a few more items from SPARK-1802 -- The first commit touches up a few similar problems remaining with the YARN profile. I think this is worth cherry-picking. The second commit is more of the same for hadoop-client, although the fix is a little more complex. It may or may not be worth bothering with. Author: Sean Owen Closes #746 from srowen/SPARK-1802.2 and squashes the following commits: 52aeb41 [Sean Owen] Add more commons-logging, servlet excludes to avoid conflicts in assembly when building for YARN --- pom.xml | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/pom.xml b/pom.xml index 3554efafb976a..4d4c5f63e44a3 100644 --- a/pom.xml +++ b/pom.xml @@ -583,6 +583,10 @@ org.jboss.netty netty + + commons-logging + commons-logging + @@ -606,6 +610,10 @@ javax.servlet servlet-api + + commons-logging + commons-logging + @@ -625,6 +633,14 @@ org.jboss.netty netty + + javax.servlet + servlet-api + + + commons-logging + commons-logging + @@ -648,6 +664,10 @@ javax.servlet servlet-api + + commons-logging + commons-logging + From beb9cbaca6dbbcaba77a34df692dc2e56b3e0638 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 12 May 2014 17:39:40 -0700 Subject: [PATCH 324/641] [SPARK-1736] Spark submit for Windows Tested on Windows 7. Author: Andrew Or Closes #745 from andrewor14/windows-submit and squashes the following commits: c0b58fb [Andrew Or] Allow spaces in parameters 162e54d [Andrew Or] Merge branch 'master' of github.com:apache/spark into windows-submit 91597ce [Andrew Or] Make spark-shell.cmd use spark-submit.cmd af6fd29 [Andrew Or] Add spark submit for Windows --- bin/spark-shell.cmd | 5 ++-- bin/spark-submit.cmd | 56 ++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 58 insertions(+), 3 deletions(-) create mode 100644 bin/spark-submit.cmd diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd index 99799128eb734..ca0c722c926f5 100755 --- a/bin/spark-shell.cmd +++ b/bin/spark-shell.cmd @@ -17,7 +17,6 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem -rem Find the path of sbin -set BIN=%~dp0..\bin\ +set SPARK_HOME=%~dp0.. -cmd /V /E /C %BIN%spark-class2.cmd org.apache.spark.repl.Main %* +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-internal %* --class org.apache.spark.repl.Main diff --git a/bin/spark-submit.cmd b/bin/spark-submit.cmd new file mode 100644 index 0000000000000..6eb702ed8c561 --- /dev/null +++ b/bin/spark-submit.cmd @@ -0,0 +1,56 @@ +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +set SPARK_HOME=%~dp0.. +set ORIG_ARGS=%* + +rem Clear the values of all variables used +set DEPLOY_MODE= +set DRIVER_MEMORY= +set SPARK_SUBMIT_LIBRARY_PATH= +set SPARK_SUBMIT_CLASSPATH= +set SPARK_SUBMIT_OPTS= +set SPARK_DRIVER_MEMORY= + +:loop +if [%1] == [] goto continue + if [%1] == [--deploy-mode] ( + set DEPLOY_MODE=%2 + ) else if [%1] == [--driver-memory] ( + set DRIVER_MEMORY=%2 + ) else if [%1] == [--driver-library-path] ( + set SPARK_SUBMIT_LIBRARY_PATH=%2 + ) else if [%1] == [--driver-class-path] ( + set SPARK_SUBMIT_CLASSPATH=%2 + ) else if [%1] == [--driver-java-options] ( + set SPARK_SUBMIT_OPTS=%2 + ) + shift +goto loop +:continue + +if [%DEPLOY_MODE%] == [] ( + set DEPLOY_MODE=client +) + +if not [%DRIVER_MEMORY%] == [] if [%DEPLOY_MODE%] == [client] ( + set SPARK_DRIVER_MEMORY=%DRIVER_MEMORY% +) + +cmd /V /E /C %SPARK_HOME%\bin\spark-class.cmd org.apache.spark.deploy.SparkSubmit %ORIG_ARGS% From 3e13b8c0bd46894392bd0fbd3e86dd55b8103858 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 15:40:48 -0700 Subject: [PATCH 325/641] Adding hadoop-2.2 profile to the build --- dev/create-release/create-release.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index b7a0a01c3bad2..c4e74990860b6 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -52,14 +52,14 @@ if [[ ! "$@" =~ --package-only ]]; then -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ --batch-mode release:prepare mvn -DskipTests \ -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ release:perform cd .. From 2f1a3373583f9b34a121236c25f5142ba8729546 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 12 May 2014 18:40:30 -0700 Subject: [PATCH 326/641] [SQL] Make Hive Metastore conversion functions publicly visible. I need this to be public for the implementation of SharkServer2. However, I think this functionality is generally useful and should be pretty stable. Author: Michael Armbrust Closes #750 from marmbrus/metastoreTypes and squashes the following commits: f51b62e [Michael Armbrust] Make Hive Metastore conversion functions publicly visible. --- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index ba837a274c51c..fa30f596f8c67 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, Catalog} import org.apache.spark.sql.catalyst.expressions._ @@ -172,7 +173,12 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with override def unregisterAllTables() = {} } -private[hive] object HiveMetastoreTypes extends RegexParsers { +/** + * :: DeveloperApi :: + * Provides conversions between Spark SQL data types and Hive Metastore types. + */ +@DeveloperApi +object HiveMetastoreTypes extends RegexParsers { protected lazy val primitiveType: Parser[DataType] = "string" ^^^ StringType | "float" ^^^ FloatType | From a5150d199ca97ab2992bc2bb221a3ebf3d3450ba Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Mon, 12 May 2014 18:46:28 -0700 Subject: [PATCH 327/641] Typo: resond -> respond Author: Andrew Ash Closes #743 from ash211/patch-4 and squashes the following commits: c959f3b [Andrew Ash] Typo: resond -> respond --- .../spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index ff8356f67681e..f08b19e6782e3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -175,7 +175,7 @@ private[spark] class MesosSchedulerBackend( override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo) {} /** - * Method called by Mesos to offer resources on slaves. We resond by asking our active task sets + * Method called by Mesos to offer resources on slaves. We respond by asking our active task sets * for tasks in order of priority. We fill each node with tasks in a round-robin manner so that * tasks are balanced across the cluster. */ From 5c2275d6e4639946fd11ff6403338c8a9ade3d1e Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Mon, 12 May 2014 19:20:24 -0700 Subject: [PATCH 328/641] L-BFGS Documentation Documentation for L-BFGS, and an example of training binary L2 logistic regression using L-BFGS. Author: DB Tsai Closes #702 from dbtsai/dbtsai-lbfgs-doc and squashes the following commits: 0712215 [DB Tsai] Update 38fdfa1 [DB Tsai] Removed extra empty line 5745b64 [DB Tsai] Update again e9e418e [DB Tsai] Update 7381521 [DB Tsai] L-BFGS Documentation --- docs/mllib-optimization.md | 120 +++++++++++++++++++++++++++++++++++-- 1 file changed, 116 insertions(+), 4 deletions(-) diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index bec3912b55dc7..aa0dec2130593 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -28,7 +28,6 @@ title: MLlib - Optimization ## Mathematical description ### Gradient descent - The simplest method to solve optimization problems of the form `$\min_{\wv \in\R^d} \; f(\wv)$` is [gradient descent](http://en.wikipedia.org/wiki/Gradient_descent). Such first-order optimization methods (including gradient descent and stochastic variants @@ -128,10 +127,19 @@ is sampled, i.e. `$|S|=$ miniBatchFraction $\cdot n = 1$`, then the algorithm is standard SGD. In that case, the step direction depends from the uniformly random sampling of the point. - +### Limited-memory BFGS (L-BFGS) +[L-BFGS](http://en.wikipedia.org/wiki/Limited-memory_BFGS) is an optimization +algorithm in the family of quasi-Newton methods to solve the optimization problems of the form +`$\min_{\wv \in\R^d} \; f(\wv)$`. The L-BFGS method approximates the objective function locally as a +quadratic without evaluating the second partial derivatives of the objective function to construct the +Hessian matrix. The Hessian matrix is approximated by previous gradient evaluations, so there is no +vertical scalability issue (the number of training features) when computing the Hessian matrix +explicitly in Newton's method. As a result, L-BFGS often achieves rapider convergence compared with +other first-order optimization. ## Implementation in MLlib +### Gradient descent and stochastic gradient descent Gradient descent methods including stochastic subgradient descent (SGD) as included as a low-level primitive in `MLlib`, upon which various ML algorithms are developed, see the @@ -142,12 +150,12 @@ The SGD method [GradientDescent.runMiniBatchSGD](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) has the following parameters: -* `gradient` is a class that computes the stochastic gradient of the function +* `Gradient` is a class that computes the stochastic gradient of the function being optimized, i.e., with respect to a single training example, at the current parameter value. MLlib includes gradient classes for common loss functions, e.g., hinge, logistic, least-squares. The gradient class takes as input a training example, its label, and the current parameter value. -* `updater` is a class that performs the actual gradient descent step, i.e. +* `Updater` is a class that performs the actual gradient descent step, i.e. updating the weights in each iteration, for a given gradient of the loss part. The updater is also responsible to perform the update from the regularization part. MLlib includes updaters for cases without regularization, as well as @@ -163,3 +171,107 @@ each iteration, to compute the gradient direction. Available algorithms for gradient descent: * [GradientDescent.runMiniBatchSGD](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) + +### L-BFGS +L-BFGS is currently only a low-level optimization primitive in `MLlib`. If you want to use L-BFGS in various +ML algorithms such as Linear Regression, and Logistic Regression, you have to pass the gradient of objective +function, and updater into optimizer yourself instead of using the training APIs like +[LogisticRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD). +See the example below. It will be addressed in the next release. + +The L1 regularization by using +[L1Updater](api/mllib/index.html#org.apache.spark.mllib.optimization.L1Updater) will not work since the +soft-thresholding logic in L1Updater is designed for gradient descent. See the developer's note. + +The L-BFGS method +[LBFGS.runLBFGS](api/scala/index.html#org.apache.spark.mllib.optimization.LBFGS) +has the following parameters: + +* `Gradient` is a class that computes the gradient of the objective function +being optimized, i.e., with respect to a single training example, at the +current parameter value. MLlib includes gradient classes for common loss +functions, e.g., hinge, logistic, least-squares. The gradient class takes as +input a training example, its label, and the current parameter value. +* `Updater` is a class that computes the gradient and loss of objective function +of the regularization part for L-BFGS. MLlib includes updaters for cases without +regularization, as well as L2 regularizer. +* `numCorrections` is the number of corrections used in the L-BFGS update. 10 is +recommended. +* `maxNumIterations` is the maximal number of iterations that L-BFGS can be run. +* `regParam` is the regularization parameter when using regularization. + +The `return` is a tuple containing two elements. The first element is a column matrix +containing weights for every feature, and the second element is an array containing +the loss computed for every iteration. + +Here is an example to train binary logistic regression with L2 regularization using +L-BFGS optimizer. +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.classification.LogisticRegressionModel + +val data = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") +val numFeatures = data.take(1)(0).features.size + +// Split data into training (60%) and test (40%). +val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) + +// Append 1 into the training data as intercept. +val training = splits(0).map(x => (x.label, MLUtils.appendBias(x.features))).cache() + +val test = splits(1) + +// Run training algorithm to build the model +val numCorrections = 10 +val convergenceTol = 1e-4 +val maxNumIterations = 20 +val regParam = 0.1 +val initialWeightsWithIntercept = Vectors.dense(new Array[Double](numFeatures + 1)) + +val (weightsWithIntercept, loss) = LBFGS.runLBFGS( + training, + new LogisticGradient(), + new SquaredL2Updater(), + numCorrections, + convergenceTol, + maxNumIterations, + regParam, + initialWeightsWithIntercept) + +val model = new LogisticRegressionModel( + Vectors.dense(weightsWithIntercept.toArray.slice(0, weightsWithIntercept.size - 1)), + weightsWithIntercept(weightsWithIntercept.size - 1)) + +// Clear the default threshold. +model.clearThreshold() + +// Compute raw scores on the test set. +val scoreAndLabels = test.map { point => + val score = model.predict(point.features) + (score, point.label) +} + +// Get evaluation metrics. +val metrics = new BinaryClassificationMetrics(scoreAndLabels) +val auROC = metrics.areaUnderROC() + +println("Loss of each step in training process") +loss.foreach(println) +println("Area under ROC = " + auROC) +{% endhighlight %} + +#### Developer's note +Since the Hessian is constructed approximately from previous gradient evaluations, +the objective function can not be changed during the optimization process. +As a result, Stochastic L-BFGS will not work naively by just using miniBatch; +therefore, we don't provide this until we have better understanding. + +* `Updater` is a class originally designed for gradient decent which computes +the actual gradient descent step. However, we're able to take the gradient and +loss of objective function of regularization for L-BFGS by ignoring the part of logic +only for gradient decent such as adaptive step size stuff. We will refactorize +this into regularizer to replace updater to separate the logic between +regularization and step update later. \ No newline at end of file From 9cf9f18973840f7287f7cfa5ce90efed3225bb30 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 12 May 2014 19:21:06 -0700 Subject: [PATCH 329/641] Modify a typo in monitoring.md As I mentioned in SPARK-1765, there is a word 'JXM' in monitoring.md. I think it's typo for 'JMX'. Author: Kousuke Saruta Closes #698 from sarutak/SPARK-1765 and squashes the following commits: bae9843 [Kousuke Saruta] modified a typoe in monitoring.md --- docs/monitoring.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/monitoring.md b/docs/monitoring.md index 6f35fc37c4075..fffc58ac39230 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -156,7 +156,7 @@ Each instance can report to zero or more _sinks_. Sinks are contained in the * `ConsoleSink`: Logs metrics information to the console. * `CSVSink`: Exports metrics data to CSV files at regular intervals. -* `JmxSink`: Registers metrics for viewing in a JXM console. +* `JmxSink`: Registers metrics for viewing in a JMX console. * `MetricsServlet`: Adds a servlet within the existing Spark UI to serve metrics data as JSON data. * `GraphiteSink`: Sends metrics to a Graphite node. From 156df87e7ca0e6cda2cc970ecd1466ce06f7576f Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Mon, 12 May 2014 19:23:39 -0700 Subject: [PATCH 330/641] SPARK-1757 Failing test for saving null primitives with .saveAsParquetFile() https://issues.apache.org/jira/browse/SPARK-1757 The first test succeeds, but the second test fails with exception: ``` [info] - save and load case class RDD with Nones as parquet *** FAILED *** (14 milliseconds) [info] java.lang.RuntimeException: Unsupported datatype StructType(List()) [info] at scala.sys.package$.error(package.scala:27) [info] at org.apache.spark.sql.parquet.ParquetTypesConverter$.fromDataType(ParquetRelation.scala:201) [info] at org.apache.spark.sql.parquet.ParquetTypesConverter$$anonfun$1.apply(ParquetRelation.scala:235) [info] at org.apache.spark.sql.parquet.ParquetTypesConverter$$anonfun$1.apply(ParquetRelation.scala:235) [info] at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) [info] at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) [info] at scala.collection.immutable.List.foreach(List.scala:318) [info] at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) [info] at scala.collection.AbstractTraversable.map(Traversable.scala:105) [info] at org.apache.spark.sql.parquet.ParquetTypesConverter$.convertFromAttributes(ParquetRelation.scala:234) [info] at org.apache.spark.sql.parquet.ParquetTypesConverter$.writeMetaData(ParquetRelation.scala:267) [info] at org.apache.spark.sql.parquet.ParquetRelation$.createEmpty(ParquetRelation.scala:143) [info] at org.apache.spark.sql.parquet.ParquetRelation$.create(ParquetRelation.scala:122) [info] at org.apache.spark.sql.execution.SparkStrategies$ParquetOperations$.apply(SparkStrategies.scala:139) [info] at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) [info] at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) [info] at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) [info] at org.apache.spark.sql.catalyst.planning.QueryPlanner.apply(QueryPlanner.scala:59) [info] at org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan$lzycompute(SQLContext.scala:264) [info] at org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan(SQLContext.scala:264) [info] at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan$lzycompute(SQLContext.scala:265) [info] at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan(SQLContext.scala:265) [info] at org.apache.spark.sql.SQLContext$QueryExecution.toRdd$lzycompute(SQLContext.scala:268) [info] at org.apache.spark.sql.SQLContext$QueryExecution.toRdd(SQLContext.scala:268) [info] at org.apache.spark.sql.SchemaRDDLike$class.saveAsParquetFile(SchemaRDDLike.scala:66) [info] at org.apache.spark.sql.SchemaRDD.saveAsParquetFile(SchemaRDD.scala:98) ``` Author: Andrew Ash Author: Michael Armbrust Closes #690 from ash211/rdd-parquet-save and squashes the following commits: 747a0b9 [Andrew Ash] Merge pull request #1 from marmbrus/pr/690 54bd00e [Michael Armbrust] Need to put Option first since Option <: Seq. 8f3f281 [Andrew Ash] SPARK-1757 Add failing test for saving SparkSQL Schemas with Option[?] fields as parquet --- .../spark/sql/catalyst/ScalaReflection.scala | 6 +-- .../spark/sql/parquet/ParquetQuerySuite.scala | 44 +++++++++++++++++++ 2 files changed, 47 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 792ef6cee6f5d..196695a0a188f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -41,6 +41,9 @@ object ScalaReflection { /** Returns a catalyst DataType for the given Scala Type using reflection. */ def schemaFor(tpe: `Type`): DataType = tpe match { + case t if t <:< typeOf[Option[_]] => + val TypeRef(_, _, Seq(optType)) = t + schemaFor(optType) case t if t <:< typeOf[Product] => val params = t.member("": TermName).asMethod.paramss StructType( @@ -59,9 +62,6 @@ object ScalaReflection { case t if t <:< typeOf[String] => StringType case t if t <:< typeOf[Timestamp] => TimestampType case t if t <:< typeOf[BigDecimal] => DecimalType - case t if t <:< typeOf[Option[_]] => - val TypeRef(_, _, Seq(optType)) = t - schemaFor(optType) case t if t <:< typeOf[java.lang.Integer] => IntegerType case t if t <:< typeOf[java.lang.Long] => LongType case t if t <:< typeOf[java.lang.Double] => DoubleType diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index d9c9b9a076ab9..ff1677eb8a480 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -42,6 +42,20 @@ import org.apache.spark.sql.test.TestSQLContext._ case class TestRDDEntry(key: Int, value: String) +case class NullReflectData( + intField: java.lang.Integer, + longField: java.lang.Long, + floatField: java.lang.Float, + doubleField: java.lang.Double, + booleanField: java.lang.Boolean) + +case class OptionalReflectData( + intField: Option[Int], + longField: Option[Long], + floatField: Option[Float], + doubleField: Option[Double], + booleanField: Option[Boolean]) + class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { import TestData._ TestData // Load test data tables. @@ -195,5 +209,35 @@ class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { Utils.deleteRecursively(ParquetTestData.testDir) ParquetTestData.writeFile() } + + test("save and load case class RDD with nulls as parquet") { + val data = NullReflectData(null, null, null, null, null) + val rdd = sparkContext.parallelize(data :: Nil) + + val file = getTempFilePath("parquet") + val path = file.toString + rdd.saveAsParquetFile(path) + val readFile = parquetFile(path) + + val rdd_saved = readFile.collect() + assert(rdd_saved(0) === Seq.fill(5)(null)) + Utils.deleteRecursively(file) + assert(true) + } + + test("save and load case class RDD with Nones as parquet") { + val data = OptionalReflectData(null, null, null, null, null) + val rdd = sparkContext.parallelize(data :: Nil) + + val file = getTempFilePath("parquet") + val path = file.toString + rdd.saveAsParquetFile(path) + val readFile = parquetFile(path) + + val rdd_saved = readFile.collect() + assert(rdd_saved(0) === Seq.fill(5)(null)) + Utils.deleteRecursively(file) + assert(true) + } } From ba96bb3d591130075763706526f86fb2aaffa3ae Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 12 May 2014 19:42:35 -0700 Subject: [PATCH 331/641] [SPARK-1780] Non-existent SPARK_DAEMON_OPTS is lurking around What they really mean is SPARK_DAEMON_***JAVA***_OPTS Author: Andrew Or Closes #751 from andrewor14/spark-daemon-opts and squashes the following commits: 70c41f9 [Andrew Or] SPARK_DAEMON_OPTS -> SPARK_DAEMON_JAVA_OPTS --- conf/spark-env.sh.template | 2 +- core/src/main/scala/org/apache/spark/SparkConf.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index f906be611a931..4479e1e34cd4a 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -39,5 +39,5 @@ # - SPARK_WORKER_DIR, to set the working directory of worker processes # - SPARK_WORKER_OPTS, to set config properties only for the worker (e.g. "-Dx=y") # - SPARK_HISTORY_OPTS, to set config properties only for the history server (e.g. "-Dx=y") -# - SPARK_DAEMON_OPTS, to set config properties for all daemons (e.g. "-Dx=y") +# - SPARK_DAEMON_JAVA_OPTS, to set config properties for all daemons (e.g. "-Dx=y") # - SPARK_PUBLIC_DNS, to set the public dns name of the master or workers diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index bd21fdc5a18e4..800616622d7bf 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -247,7 +247,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application | - ./spark-submit with --driver-java-options to set -X options for a driver | - spark.executor.extraJavaOptions to set -X options for executors - | - SPARK_DAEMON_OPTS to set java options for standalone daemons (i.e. master, worker) + | - SPARK_DAEMON_JAVA_OPTS to set java options for standalone daemons (master or worker) """.stripMargin logError(error) From 2ffd1eafd28635dcecc0ac738d4a62c05d740925 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 12 May 2014 19:44:14 -0700 Subject: [PATCH 332/641] [SPARK-1753 / 1773 / 1814] Update outdated docs for spark-submit, YARN, standalone etc. YARN - SparkPi was updated to not take in master as an argument; we should update the docs to reflect that. - The default YARN build guide should be in maven, not sbt. - This PR also adds a paragraph on steps to debug a YARN application. Standalone - Emphasize spark-submit more. Right now it's one small paragraph preceding the legacy way of launching through `org.apache.spark.deploy.Client`. - The way we set configurations / environment variables according to the old docs is outdated. This needs to reflect changes introduced by the Spark configuration changes we made. In general, this PR also adds a little more documentation on the new spark-shell, spark-submit, spark-defaults.conf etc here and there. Author: Andrew Or Closes #701 from andrewor14/yarn-docs and squashes the following commits: e2c2312 [Andrew Or] Merge in changes in #752 (SPARK-1814) 25cfe7b [Andrew Or] Merge in the warning from SPARK-1753 a8c39c5 [Andrew Or] Minor changes 336bbd9 [Andrew Or] Tabs -> spaces 4d9d8f7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 041017a [Andrew Or] Abstract Spark submit documentation to cluster-overview.html 3cc0649 [Andrew Or] Detail how to set configurations + remove legacy instructions 5b7140a [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 85a51fc [Andrew Or] Update run-example, spark-shell, configuration etc. c10e8c7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 381fe32 [Andrew Or] Update docs for standalone mode 757c184 [Andrew Or] Add a note about the requirements for the debugging trick f8ca990 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 924f04c [Andrew Or] Revert addition of --deploy-mode d5fe17b [Andrew Or] Update the YARN docs --- conf/spark-defaults.conf.template | 3 +- conf/spark-env.sh.template | 4 +- docs/building-with-maven.md | 7 +++ docs/cluster-overview.md | 73 +++++++++++++++--------- docs/configuration.md | 64 +++++++++++++-------- docs/hadoop-third-party-distributions.md | 14 +++-- docs/index.md | 34 +++++++---- docs/java-programming-guide.md | 5 +- docs/python-programming-guide.md | 2 +- docs/quick-start.md | 4 +- docs/running-on-yarn.md | 15 +++-- docs/scala-programming-guide.md | 13 +++-- docs/spark-standalone.md | 71 ++++++++++------------- 13 files changed, 184 insertions(+), 125 deletions(-) diff --git a/conf/spark-defaults.conf.template b/conf/spark-defaults.conf.template index f840ff681d019..2779342769c14 100644 --- a/conf/spark-defaults.conf.template +++ b/conf/spark-defaults.conf.template @@ -2,6 +2,7 @@ # This is useful for setting default environmental settings. # Example: -# spark.master spark://master:7077 +# spark.master spark://master:7077 # spark.eventLog.enabled true # spark.eventLog.dir hdfs://namenode:8021/directory +# spark.serializer org.apache.spark.serializer.KryoSerializer diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 4479e1e34cd4a..f8ffbf64278fb 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -30,11 +30,11 @@ # Options for the daemons used in the standalone deploy mode: # - SPARK_MASTER_IP, to bind the master to a different IP address or hostname -# - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports +# - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports for the master # - SPARK_MASTER_OPTS, to set config properties only for the master (e.g. "-Dx=y") # - SPARK_WORKER_CORES, to set the number of cores to use on this machine # - SPARK_WORKER_MEMORY, to set how much total memory workers have to give executors (e.g. 1000m, 2g) -# - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT +# - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT, to use non-default ports for the worker # - SPARK_WORKER_INSTANCES, to set the number of worker processes per node # - SPARK_WORKER_DIR, to set the working directory of worker processes # - SPARK_WORKER_OPTS, to set config properties only for the worker (e.g. "-Dx=y") diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index b6dd553bbe06b..8b44535d82404 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -129,6 +129,13 @@ Java 8 tests are run when -Pjava8-tests profile is enabled, they will run in spi For these tests to run your system must have a JDK 8 installation. If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. +## Building for PySpark on YARN ## + +PySpark on YARN is only supported if the jar is built with maven. Further, there is a known problem +with building this assembly jar on Red Hat based operating systems (see SPARK-1753). If you wish to +run PySpark on a YARN cluster with Red Hat installed, we recommend that you build the jar elsewhere, +then ship it over to the cluster. We are investigating the exact cause for this. + ## Packaging without Hadoop dependencies for deployment on YARN ## The assembly jar produced by "mvn package" will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with yarn.application.classpath. The "hadoop-provided" profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 162c415b5883e..f05a755de7fec 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -66,62 +66,76 @@ script as shown here while passing your jar. For Python, you can use the `pyFiles` argument of SparkContext or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be distributed. -### Launching Applications with ./bin/spark-submit +### Launching Applications with Spark submit Once a user application is bundled, it can be launched using the `spark-submit` script located in the bin directory. This script takes care of setting up the classpath with Spark and its -dependencies, and can support different cluster managers and deploy modes that Spark supports. -It's usage is +dependencies, and can support different cluster managers and deploy modes that Spark supports: - ./bin/spark-submit --class path.to.your.Class [options] [app options] + ./bin/spark-submit \ + --class + --master \ + --deploy-mode \ + ... // other options + + [application-arguments] -When calling `spark-submit`, `[app options]` will be passed along to your application's -main class. To enumerate all options available to `spark-submit` run it with -the `--help` flag. Here are a few examples of common options: + main-class: The entry point for your application (e.g. org.apache.spark.examples.SparkPi) + master-url: The URL of the master node (e.g. spark://23.195.26.187:7077) + deploy-mode: Whether to deploy this application within the cluster or from an external client (e.g. client) + application-jar: Path to a bundled jar including your application and all dependencies. The URL must be globally visible inside of your cluster, for instance, an `hdfs://` path or a `file://` path that is present on all nodes. + application-arguments: Space delimited arguments passed to the main method of , if any + +To enumerate all options available to `spark-submit` run it with the `--help` flag. Here are a few +examples of common options: {% highlight bash %} # Run application locally ./bin/spark-submit \ - --class my.main.ClassName + --class org.apache.spark.examples.SparkPi --master local[8] \ - my-app.jar + /path/to/examples.jar \ + 100 # Run on a Spark standalone cluster ./bin/spark-submit \ - --class my.main.ClassName - --master spark://mycluster:7077 \ + --class org.apache.spark.examples.SparkPi + --master spark://207.184.161.138:7077 \ --executor-memory 20G \ --total-executor-cores 100 \ - my-app.jar + /path/to/examples.jar \ + 1000 # Run on a YARN cluster -HADOOP_CONF_DIR=XX /bin/spark-submit \ - --class my.main.ClassName +HADOOP_CONF_DIR=XX ./bin/spark-submit \ + --class org.apache.spark.examples.SparkPi --master yarn-cluster \ # can also be `yarn-client` for client mode --executor-memory 20G \ --num-executors 50 \ - my-app.jar + /path/to/examples.jar \ + 1000 {% endhighlight %} ### Loading Configurations from a File -The `spark-submit` script can load default `SparkConf` values from a properties file and pass them -onto your application. By default it will read configuration options from -`conf/spark-defaults.conf`. Any values specified in the file will be passed on to the -application when run. They can obviate the need for certain flags to `spark-submit`: for -instance, if `spark.master` property is set, you can safely omit the +The `spark-submit` script can load default [Spark configuration values](configuration.html) from a +properties file and pass them on to your application. By default it will read configuration options +from `conf/spark-defaults.conf`. For more detail, see the section on +[loading default configurations](configuration.html#loading-default-configurations). + +Loading default Spark configurations this way can obviate the need for certain flags to +`spark-submit`. For instance, if the `spark.master` property is set, you can safely omit the `--master` flag from `spark-submit`. In general, configuration values explicitly set on a -`SparkConf` take the highest precedence, then flags passed to `spark-submit`, then values -in the defaults file. +`SparkConf` take the highest precedence, then flags passed to `spark-submit`, then values in the +defaults file. -If you are ever unclear where configuration options are coming from. fine-grained debugging -information can be printed by adding the `--verbose` option to `./spark-submit`. +If you are ever unclear where configuration options are coming from, you can print out fine-grained +debugging information by running `spark-submit` with the `--verbose` option. ### Advanced Dependency Management -When using `./bin/spark-submit` the app jar along with any jars included with the `--jars` option -will be automatically transferred to the cluster. `--jars` can also be used to distribute .egg and .zip -libraries for Python to executors. Spark uses the following URL scheme to allow different -strategies for disseminating jars: +When using `spark-submit`, the application jar along with any jars included with the `--jars` option +will be automatically transferred to the cluster. Spark uses the following URL scheme to allow +different strategies for disseminating jars: - **file:** - Absolute paths and `file:/` URIs are served by the driver's HTTP file server, and every executor pulls the file from the driver HTTP server. @@ -135,6 +149,9 @@ This can use up a significant amount of space over time and will need to be clea is handled automatically, and with Spark standalone, automatic cleanup can be configured with the `spark.worker.cleanup.appDataTtl` property. +For python, the equivalent `--py-files` option can be used to distribute .egg and .zip libraries +to executors. + # Monitoring Each driver program has a web UI, typically on port 4040, that displays information about running diff --git a/docs/configuration.md b/docs/configuration.md index 5b034e3cb3d47..2eed96f704a4f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -5,9 +5,9 @@ title: Spark Configuration Spark provides three locations to configure the system: -* [Spark properties](#spark-properties) control most application parameters and can be set by passing - a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object to SparkContext, or through Java - system properties. +* [Spark properties](#spark-properties) control most application parameters and can be set by + passing a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object to SparkContext, + or through the `conf/spark-defaults.conf` properties file. * [Environment variables](#environment-variables) can be used to set per-machine settings, such as the IP address, through the `conf/spark-env.sh` script on each node. * [Logging](#configuring-logging) can be configured through `log4j.properties`. @@ -15,25 +15,41 @@ Spark provides three locations to configure the system: # Spark Properties -Spark properties control most application settings and are configured separately for each application. -The preferred way to set them is by passing a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) -class to your SparkContext constructor. -Alternatively, Spark will also load them from Java system properties, for compatibility with old versions -of Spark. - -SparkConf lets you configure most of the common properties to initialize a cluster (e.g., master URL and -application name), as well as arbitrary key-value pairs through the `set()` method. For example, we could -initialize an application as follows: +Spark properties control most application settings and are configured separately for each +application. The preferred way is to set them through +[SparkConf](api/scala/index.html#org.apache.spark.SparkConf) and passing it as an argument to your +SparkContext. SparkConf allows you to configure most of the common properties to initialize a +cluster (e.g. master URL and application name), as well as arbitrary key-value pairs through the +`set()` method. For example, we could initialize an application as follows: {% highlight scala %} -val conf = new SparkConf(). - setMaster("local"). - setAppName("My application"). - set("spark.executor.memory", "1g") +val conf = new SparkConf + .setMaster("local") + .setAppName("CountingSheep") + .set("spark.executor.memory", "1g") val sc = new SparkContext(conf) {% endhighlight %} -Most of the properties control internal settings that have reasonable default values. However, +## Loading Default Configurations + +In the case of `spark-shell`, a SparkContext has already been created for you, so you cannot control +the configuration properties through SparkConf. However, you can still set configuration properties +through a default configuration file. By default, `spark-shell` (and more generally `spark-submit`) +will read configuration options from `conf/spark-defaults.conf`, in which each line consists of a +key and a value separated by whitespace. For example, + + spark.master spark://5.6.7.8:7077 + spark.executor.memory 512m + spark.eventLog.enabled true + spark.serializer org.apache.spark.serializer.KryoSerializer + +Any values specified in the file will be passed on to the application, and merged with those +specified through SparkConf. If the same configuration property exists in both `spark-defaults.conf` +and SparkConf, then the latter will take precedence as it is the most application-specific. + +## All Configuration Properties + +Most of the properties that control internal settings have reasonable default values. However, there are at least five properties that you will commonly want to control: @@ -101,9 +117,9 @@ Apart from these, the following properties are also available, and may be useful @@ -696,7 +712,9 @@ Apart from these, the following properties are also available, and may be useful ## Viewing Spark Properties The application web UI at `http://:4040` lists Spark properties in the "Environment" tab. -This is a useful place to check to make sure that your properties have been set correctly. +This is a useful place to check to make sure that your properties have been set correctly. Note +that only values explicitly specified through either `spark-defaults.conf` or SparkConf will +appear. For all other configuration properties, you can assume the default value is used. # Environment Variables @@ -714,8 +732,8 @@ The following variables can be set in `spark-env.sh`: * `PYSPARK_PYTHON`, the Python binary to use for PySpark * `SPARK_LOCAL_IP`, to configure which IP address of the machine to bind to. * `SPARK_PUBLIC_DNS`, the hostname your Spark program will advertise to other machines. -* Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores - to use on each machine and maximum memory. +* Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), + such as number of cores to use on each machine and maximum memory. Since `spark-env.sh` is a shell script, some of these can be set programmatically -- for example, you might compute `SPARK_LOCAL_IP` by looking up the IP of a specific network interface. diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index 454877a7fa8a5..a0aeab5727bde 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -9,12 +9,14 @@ with these distributions: # Compile-time Hadoop Version -When compiling Spark, you'll need to -[set the SPARK_HADOOP_VERSION flag](index.html#a-note-about-hadoop-versions): +When compiling Spark, you'll need to specify the Hadoop version by defining the `hadoop.version` +property. For certain versions, you will need to specify additional profiles. For more detail, +see the guide on [building with maven](building-with-maven.html#specifying-the-hadoop-version): - SPARK_HADOOP_VERSION=1.0.4 sbt/sbt assembly + mvn -Dhadoop.version=1.0.4 -DskipTests clean package + mvn -Phadoop-2.2 -Dhadoop.version=2.2.0 -DskipTests clean package -The table below lists the corresponding `SPARK_HADOOP_VERSION` code for each CDH/HDP release. Note that +The table below lists the corresponding `hadoop.version` code for each CDH/HDP release. Note that some Hadoop releases are binary compatible across client versions. This means the pre-built Spark distribution may "just work" without you needing to compile. That said, we recommend compiling with the _exact_ Hadoop version you are running to avoid any compatibility errors. @@ -46,6 +48,10 @@ the _exact_ Hadoop version you are running to avoid any compatibility errors.
      spark.default.parallelism
        +
      • Local mode: number of cores on the local machine
      • Mesos fine grained mode: 8
      • -
      • Local mode: core number of the local machine
      • -
      • Others: total core number of all executor nodes or 2, whichever is larger
      • +
      • Others: total number of cores on all executor nodes or 2, whichever is larger
      @@ -187,7 +203,7 @@ Apart from these, the following properties are also available, and may be useful Comma separated list of filter class names to apply to the Spark web ui. The filter should be a standard javax servlet Filter. Parameters to each filter can also be specified by setting a java system property of spark.<class name of filter>.params='param1=value1,param2=value2' - (e.g.-Dspark.ui.filters=com.test.filter1 -Dspark.com.test.filter1.params='param1=foo,param2=testing') + (e.g. -Dspark.ui.filters=com.test.filter1 -Dspark.com.test.filter1.params='param1=foo,param2=testing')
      +In SBT, the equivalent can be achieved by setting the SPARK_HADOOP_VERSION flag: + + SPARK_HADOOP_VERSION=1.0.4 sbt/sbt assembly + # Linking Applications to the Hadoop Version In addition to compiling Spark itself against the right version, you need to add a Maven dependency on that diff --git a/docs/index.md b/docs/index.md index a2f1a84371ff4..48182a27d28ae 100644 --- a/docs/index.md +++ b/docs/index.md @@ -24,21 +24,31 @@ right version of Scala from [scala-lang.org](http://www.scala-lang.org/download/ # Running the Examples and Shell -Spark comes with several sample programs. Scala, Java and Python examples are in the `examples/src/main` directory. -To run one of the Java or Scala sample programs, use `./bin/run-example ` in the top-level Spark directory -(the `bin/run-example` script sets up the appropriate paths and launches that program). -For example, try `./bin/run-example org.apache.spark.examples.SparkPi local`. -To run a Python sample program, use `./bin/pyspark `. For example, try `./bin/pyspark ./examples/src/main/python/pi.py local`. +Spark comes with several sample programs. Scala, Java and Python examples are in the +`examples/src/main` directory. To run one of the Java or Scala sample programs, use +`bin/run-example [params]` in the top-level Spark directory. (Behind the scenes, this +invokes the more general +[Spark submit script](cluster-overview.html#launching-applications-with-spark-submit) for +launching applications). For example, -Each example prints usage help when run with no parameters. + ./bin/run-example SparkPi 10 -Note that all of the sample programs take a `` parameter specifying the cluster URL -to connect to. This can be a [URL for a distributed cluster](scala-programming-guide.html#master-urls), -or `local` to run locally with one thread, or `local[N]` to run locally with N threads. You should start by using -`local` for testing. +You can also run Spark interactively through modified versions of the Scala shell. This is a +great way to learn the framework. -Finally, you can run Spark interactively through modified versions of the Scala shell (`./bin/spark-shell`) or -Python interpreter (`./bin/pyspark`). These are a great way to learn the framework. + ./bin/spark-shell --master local[2] + +The `--master` option specifies the +[master URL for a distributed cluster](scala-programming-guide.html#master-urls), or `local` to run +locally with one thread, or `local[N]` to run locally with N threads. You should start by using +`local` for testing. For a full list of options, run Spark shell with the `--help` option. + +Spark also provides a Python interface. To run an example Spark application written in Python, use +`bin/pyspark [params]`. For example, + + ./bin/pyspark examples/src/main/python/pi.py local[2] 10 + +or simply `bin/pyspark` without any arguments to run Spark interactively in a python interpreter. # Launching on a Cluster diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md index c34eb28fc06a2..943fdd9d019ff 100644 --- a/docs/java-programming-guide.md +++ b/docs/java-programming-guide.md @@ -215,7 +215,4 @@ Spark includes several sample programs using the Java API in [`examples/src/main/java`](https://github.com/apache/spark/tree/master/examples/src/main/java/org/apache/spark/examples). You can run them by passing the class name to the `bin/run-example` script included in Spark; for example: - ./bin/run-example org.apache.spark.examples.JavaWordCount - -Each example program prints usage help when run -without any arguments. + ./bin/run-example JavaWordCount README.md diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 39fb5f0c99ca3..2ce2c346d7b04 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -164,6 +164,6 @@ some example applications. PySpark also includes several sample programs in the [`examples/src/main/python` folder](https://github.com/apache/spark/tree/master/examples/src/main/python). You can run them by passing the files to `pyspark`; e.g.: - ./bin/spark-submit examples/src/main/python/wordcount.py + ./bin/spark-submit examples/src/main/python/wordcount.py local[2] README.md Each program prints usage help when run without arguments. diff --git a/docs/quick-start.md b/docs/quick-start.md index 478b790f92e17..a4d01487bb494 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -18,7 +18,9 @@ you can download a package for any version of Hadoop. ## Basics Spark's interactive shell provides a simple way to learn the API, as well as a powerful tool to analyze datasets interactively. -Start the shell by running `./bin/spark-shell` in the Spark directory. +Start the shell by running the following in the Spark directory. + + ./bin/spark-shell Spark's primary abstraction is a distributed collection of items called a Resilient Distributed Dataset (RDD). RDDs can be created from Hadoop InputFormats (such as HDFS files) or by transforming other RDDs. Let's make a new RDD from the text of the README file in the Spark source directory: diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index c563594296802..66c330fdee734 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -54,13 +54,13 @@ For example: --executor-memory 2g \ --executor-cores 1 lib/spark-examples*.jar \ - yarn-cluster 5 + 10 The above starts a YARN client program which starts the default Application Master. Then SparkPi will be run as a child thread of Application Master. The client will periodically poll the Application Master for status updates and display them in the console. The client will exit once your application has finished running. Refer to the "Viewing Logs" section below for how to see driver and executor logs. To launch a Spark application in yarn-client mode, do the same, but replace "yarn-cluster" with "yarn-client". To run spark-shell: - $ MASTER=yarn-client ./bin/spark-shell + $ ./bin/spark-shell --master yarn-client ## Adding additional jars @@ -70,9 +70,9 @@ In yarn-cluster mode, the driver runs on a different machine than the client, so --master yarn-cluster \ --jars my-other-jar.jar,my-other-other-jar.jar my-main-jar.jar - yarn-cluster 5 + app_arg1 app_arg2 -# Viewing logs +# Debugging your Application In YARN terminology, executors and application masters run inside "containers". YARN has two modes for handling container logs after an application has completed. If log aggregation is turned on (with the yarn.log-aggregation-enable config), container logs are copied to HDFS and deleted on the local machine. These logs can be viewed from anywhere on the cluster with the "yarn logs" command. @@ -82,6 +82,13 @@ will print out the contents of all log files from all containers from the given When log aggregation isn't turned on, logs are retained locally on each machine under YARN_APP_LOGS_DIR, which is usually configured to /tmp/logs or $HADOOP_HOME/logs/userlogs depending on the Hadoop version and installation. Viewing logs for a container requires going to the host that contains them and looking in this directory. Subdirectories organize log files by application ID and container ID. +To review per-container launch environment, increase yarn.nodemanager.delete.debug-delay-sec to a +large value (e.g. 36000), and then access the application cache through yarn.nodemanager.local-dirs +on the nodes on which containers are launched. This directory contains the launch script, jars, and +all environment variables used for launching each container. This process is useful for debugging +classpath problems in particular. (Note that enabling this requires admin privileges on cluster +settings and a restart of all node managers. Thus, this is not applicable to hosted clusters). + # Important notes - Before Hadoop 2.2, YARN does not support cores in container resource requests. Thus, when running against an earlier version, the numbers of cores given via command line arguments cannot be passed to YARN. Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index f25e9cca88524..3ed86e460c01c 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -56,7 +56,7 @@ The `master` parameter is a string specifying a [Spark, Mesos or YARN cluster UR to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. It's also possible to set these variables [using a configuration file](cluster-overview.html#loading-configurations-from-a-file) -which avoids hard-coding the master name in your application. +which avoids hard-coding the master url in your application. In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the @@ -74,6 +74,11 @@ Or, to also add `code.jar` to its classpath, use: $ ./bin/spark-shell --master local[4] --jars code.jar {% endhighlight %} +For a complete list of options, run Spark shell with the `--help` option. Behind the scenes, +Spark shell invokes the more general [Spark submit script](cluster-overview.html#launching-applications-with-spark-submit) +used for launching applications, and passes on all of its parameters. As a result, these two scripts +share the same parameters. + ### Master URLs The master URL passed to Spark can be in one of the following formats: @@ -98,7 +103,7 @@ cluster mode. The cluster location will be inferred based on the local Hadoop co -If no master URL is specified, the spark shell defaults to "local[*]". +If no master URL is specified, the spark shell defaults to `local[*]`. # Resilient Distributed Datasets (RDDs) @@ -432,9 +437,7 @@ res2: Int = 10 You can see some [example Spark programs](http://spark.apache.org/examples.html) on the Spark website. In addition, Spark includes several samples in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `bin/run-example` script included in Spark; for example: - ./bin/run-example org.apache.spark.examples.SparkPi - -Each example program prints usage help when run without any arguments. + ./bin/run-example SparkPi For help on optimizing your program, the [configuration](configuration.html) and [tuning](tuning.html) guides provide information on best practices. They are especially important for diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index dc7f206e03996..eb3211b6b0e4e 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -70,7 +70,7 @@ Once you've set up this file, you can launch or stop your cluster with the follo - `sbin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file. - `sbin/start-all.sh` - Starts both a master and a number of slaves as described above. - `sbin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script. -- `sbin/stop-slaves.sh` - Stops the slave instances that were started via `bin/start-slaves.sh`. +- `sbin/stop-slaves.sh` - Stops all slave instances on the machines specified in the `conf/slaves` file. - `sbin/stop-all.sh` - Stops both the master and the slaves as described above. Note that these scripts must be executed on the machine you want to run the Spark master on, not your local machine. @@ -92,12 +92,8 @@ You can optionally configure the cluster further by setting environment variable Port for the master web UI (default: 8080). - SPARK_WORKER_PORT - Start the Spark worker on a specific port (default: random). - - - SPARK_WORKER_DIR - Directory to run applications in, which will include both logs and scratch space (default: SPARK_HOME/work). + SPARK_MASTER_OPTS + Configuration properties that apply only to the master in the form "-Dx=y" (default: none). SPARK_WORKER_CORES @@ -107,6 +103,10 @@ You can optionally configure the cluster further by setting environment variable SPARK_WORKER_MEMORY Total amount of memory to allow Spark applications to use on the machine, e.g. 1000m, 2g (default: total memory minus 1 GB); note that each application's individual memory is configured using its spark.executor.memory property. + + SPARK_WORKER_PORT + Start the Spark worker on a specific port (default: random). + SPARK_WORKER_WEBUI_PORT Port for the worker web UI (default: 8081). @@ -120,13 +120,25 @@ You can optionally configure the cluster further by setting environment variable or else each worker will try to use all the cores. + + SPARK_WORKER_DIR + Directory to run applications in, which will include both logs and scratch space (default: SPARK_HOME/work). + + + SPARK_WORKER_OPTS + Configuration properties that apply only to the worker in the form "-Dx=y" (default: none). + SPARK_DAEMON_MEMORY Memory to allocate to the Spark master and worker daemons themselves (default: 512m). SPARK_DAEMON_JAVA_OPTS - JVM options for the Spark master and worker daemons themselves (default: none). + JVM options for the Spark master and worker daemons themselves in the form "-Dx=y" (default: none). + + + SPARK_PUBLIC_DNS + The public DNS name of the Spark master and workers (default: none). @@ -148,38 +160,17 @@ You can also pass an option `--cores ` to control the number of cores # Launching Compiled Spark Applications -Spark supports two deploy modes. Spark applications may run with the driver inside the client process or entirely inside the cluster. - -The spark-submit script described in the [cluster mode overview](cluster-overview.html) provides the most straightforward way to submit a compiled Spark application to the cluster in either deploy mode. For info on the lower-level invocations used to launch an app inside the cluster, read ahead. - -## Launching Applications Inside the Cluster - - ./bin/spark-class org.apache.spark.deploy.Client launch - [client-options] \ - \ - [application-options] - - cluster-url: The URL of the master node. - application-jar-url: Path to a bundled jar including your application and all dependencies. Currently, the URL must be globally visible inside of your cluster, for instance, an `hdfs://` path or a `file://` path that is present on all nodes. - main-class: The entry point for your application. - - Client Options: - --memory (amount of memory, in MB, allocated for your driver program) - --cores (number of cores allocated for your driver program) - --supervise (whether to automatically restart your driver on application or node failure) - --verbose (prints increased logging output) - -Keep in mind that your driver program will be executed on a remote worker machine. You can control the execution environment in the following ways: - - * _Environment variables_: These will be captured from the environment in which you launch the client and applied when launching the driver program. - * _Java options_: You can add java options by setting `SPARK_JAVA_OPTS` in the environment in which you launch the submission client. - * _Dependencies_: You'll still need to call `sc.addJar` inside of your program to make your bundled application jar visible on all worker nodes. - -Once you submit a driver program, it will appear in the cluster management UI at port 8080 and -be assigned an identifier. If you'd like to prematurely terminate the program, you can do so using -the same client: +Spark supports two deploy modes: applications may run with the driver inside the client process or +entirely inside the cluster. The +[Spark submit script](cluster-overview.html#launching-applications-with-spark-submit) provides the +most straightforward way to submit a compiled Spark application to the cluster in either deploy +mode. - ./bin/spark-class org.apache.spark.deploy.Client kill +If your application is launched through Spark submit, then the application jar is automatically +distributed to all worker nodes. For any additional jars that your application depends on, you +should specify them through the `--jars` flag using comma as a delimiter (e.g. `--jars jar1,jar2`). +To control the application's configuration or execution environment, see +[Spark Configuration](configuration.html). # Resource Scheduling @@ -203,7 +194,7 @@ default for applications that don't set `spark.cores.max` to something less than Do this by adding the following to `conf/spark-env.sh`: {% highlight bash %} -export SPARK_JAVA_OPTS="-Dspark.deploy.defaultCores=" +export SPARK_MASTER_OPTS="-Dspark.deploy.defaultCores=" {% endhighlight %} This is useful on shared clusters where users might not have configured a maximum number of cores From 2792bd016af2a67848e6f403c4e1e05e9f3e3c2a Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 12 May 2014 20:08:30 -0700 Subject: [PATCH 333/641] SPARK-1815. SparkContext should not be marked DeveloperApi Author: Sandy Ryza Closes #753 from sryza/sandy-spark-1815 and squashes the following commits: 957a8ac [Sandy Ryza] SPARK-1815. SparkContext should not be marked DeveloperApi --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 35beff0f8ef3d..c43b4fd6d926f 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -50,7 +50,6 @@ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} /** - * :: DeveloperApi :: * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. * @@ -58,7 +57,6 @@ import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerTy * this config overrides the default configs as well as system properties. */ -@DeveloperApi class SparkContext(config: SparkConf) extends Logging { // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, From 52d905296fd75b3bff34a16cb69e604b68b3c246 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 23:02:54 -0700 Subject: [PATCH 334/641] BUILD: Add more content to make-distribution.sh. --- make-distribution.sh | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/make-distribution.sh b/make-distribution.sh index 1cc2844703fbb..7a08d6b9151c4 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -171,10 +171,22 @@ echo "Spark $VERSION built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" cp $FWDIR/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" +# Copy example sources (needed for python and SQL) +mkdir -p "$DISTDIR/examples/src/main" +cp -r $FWDIR/examples/src/main "$DISTDIR/examples/src/" + if [ "$SPARK_HIVE" == "true" ]; then cp $FWDIR/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/" fi +# Copy license and ASF files +cp "$FWDIR/LICENSE" "$DISTDIR" +cp "$FWDIR/NOTICE" "$DISTDIR" + +if [ -e $FWDIR/CHANGES.txt ]; then + cp "$FWDIR/CHANGES.txt" "$DISTDIR" +fi + # Copy other things mkdir "$DISTDIR"/conf cp "$FWDIR"/conf/*.template "$DISTDIR"/conf @@ -182,6 +194,7 @@ cp "$FWDIR"/conf/slaves "$DISTDIR"/conf cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/python" "$DISTDIR" cp -r "$FWDIR/sbin" "$DISTDIR" +cp -r "$FWDIR/ec2" "$DISTDIR" # Download and copy in tachyon, if requested if [ "$SPARK_TACHYON" == "true" ]; then From 16ffadcc4af21430b5079dc555bcd9d8cf1fa1fa Mon Sep 17 00:00:00 2001 From: William Benton Date: Tue, 13 May 2014 13:45:23 -0700 Subject: [PATCH 335/641] SPARK-571: forbid return statements in cleaned closures This patch checks top-level closure arguments to `ClosureCleaner.clean` for `return` statements and raises an exception if it finds any. This is mainly a user-friendliness addition, since programs with return statements in closure arguments will currently fail upon RDD actions with a less-than-intuitive error message. Author: William Benton Closes #717 from willb/spark-571 and squashes the following commits: c41eb7d [William Benton] Another test case for SPARK-571 30c42f4 [William Benton] Stylistic cleanups 559b16b [William Benton] Stylistic cleanups from review de13b79 [William Benton] Style fixes 295b6a5 [William Benton] Forbid return statements in closure arguments. b017c47 [William Benton] Added a test for SPARK-571 --- .../apache/spark/util/ClosureCleaner.scala | 23 ++++++++++- .../spark/util/ClosureCleanerSuite.scala | 39 ++++++++++++++++++- 2 files changed, 60 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index 2d05e09b10948..4916d9b86cca5 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.Set import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkException} private[spark] object ClosureCleaner extends Logging { // Get an ASM class reader for a given class from the JAR that loaded it @@ -108,6 +108,9 @@ private[spark] object ClosureCleaner extends Logging { val outerObjects = getOuterObjects(func) val accessedFields = Map[Class[_], Set[String]]() + + getClassReader(func.getClass).accept(new ReturnStatementFinder(), 0) + for (cls <- outerClasses) accessedFields(cls) = Set[String]() for (cls <- func.getClass :: innerClasses) @@ -180,6 +183,24 @@ private[spark] object ClosureCleaner extends Logging { } } +private[spark] +class ReturnStatementFinder extends ClassVisitor(ASM4) { + override def visitMethod(access: Int, name: String, desc: String, + sig: String, exceptions: Array[String]): MethodVisitor = { + if (name.contains("apply")) { + new MethodVisitor(ASM4) { + override def visitTypeInsn(op: Int, tp: String) { + if (op == NEW && tp.contains("scala/runtime/NonLocalReturnControl")) { + throw new SparkException("Return statements aren't allowed in Spark closures") + } + } + } + } else { + new MethodVisitor(ASM4) {} + } + } +} + private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor(ASM4) { override def visitMethod(access: Int, name: String, desc: String, diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index d7e48e633e0ee..054ef54e746a5 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.util import org.scalatest.FunSuite import org.apache.spark.LocalSparkContext._ -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, SparkException} class ClosureCleanerSuite extends FunSuite { test("closures inside an object") { @@ -50,6 +50,19 @@ class ClosureCleanerSuite extends FunSuite { val obj = new TestClassWithNesting(1) assert(obj.run() === 96) // 4 * (1+2+3+4) + 4 * (1+2+3+4) + 16 * 1 } + + test("toplevel return statements in closures are identified at cleaning time") { + val ex = intercept[SparkException] { + TestObjectWithBogusReturns.run() + } + + assert(ex.getMessage.contains("Return statements aren't allowed in Spark closures")) + } + + test("return statements from named functions nested in closures don't raise exceptions") { + val result = TestObjectWithNestedReturns.run() + assert(result == 1) + } } // A non-serializable class we create in closures to make sure that we aren't @@ -108,6 +121,30 @@ class TestClassWithoutFieldAccess { } } +object TestObjectWithBogusReturns { + def run(): Int = { + withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + // this return is invalid since it will transfer control outside the closure + nums.map {x => return 1 ; x * 2} + 1 + } + } +} + +object TestObjectWithNestedReturns { + def run(): Int = { + withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + nums.map {x => + // this return is fine since it will not transfer control outside the closure + def foo(): Int = { return 5; 1 } + foo() + } + 1 + } + } +} object TestObjectWithNesting { def run(): Int = { From d1e487473fd509f28daf28dcda856f3c2f1194ec Mon Sep 17 00:00:00 2001 From: Andrew Tulloch Date: Tue, 13 May 2014 17:31:27 -0700 Subject: [PATCH 336/641] SPARK-1791 - SVM implementation does not use threshold parameter Summary: https://issues.apache.org/jira/browse/SPARK-1791 Simple fix, and backward compatible, since - anyone who set the threshold was getting completely wrong answers. - anyone who did not set the threshold had the default 0.0 value for the threshold anyway. Test Plan: Unit test added that is verified to fail under the old implementation, and pass under the new implementation. Reviewers: CC: Author: Andrew Tulloch Closes #725 from ajtulloch/SPARK-1791-SVM and squashes the following commits: 770f55d [Andrew Tulloch] SPARK-1791 - SVM implementation does not use threshold parameter --- .../spark/mllib/classification/SVM.scala | 2 +- .../spark/mllib/classification/SVMSuite.scala | 37 +++++++++++++++++++ 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index e05213536e64a..316ecd713b715 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -65,7 +65,7 @@ class SVMModel private[mllib] ( intercept: Double) = { val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept threshold match { - case Some(t) => if (margin < 0) 0.0 else 1.0 + case Some(t) => if (margin < t) 0.0 else 1.0 case None => margin } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala index 77d6f04b32320..886c71dde3af7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala @@ -69,6 +69,43 @@ class SVMSuite extends FunSuite with LocalSparkContext { assert(numOffPredictions < input.length / 5) } + test("SVM with threshold") { + val nPoints = 10000 + + // NOTE: Intercept should be small for generating equal 0s and 1s + val A = 0.01 + val B = -1.5 + val C = 1.0 + + val testData = SVMSuite.generateSVMInput(A, Array[Double](B, C), nPoints, 42) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val svm = new SVMWithSGD().setIntercept(true) + svm.optimizer.setStepSize(1.0).setRegParam(1.0).setNumIterations(100) + + val model = svm.run(testRDD) + + val validationData = SVMSuite.generateSVMInput(A, Array[Double](B, C), nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + + // Test prediction on RDD. + + var predictions = model.predict(validationRDD.map(_.features)).collect() + assert(predictions.count(_ == 0.0) != predictions.length) + + // High threshold makes all the predictions 0.0 + model.setThreshold(10000.0) + predictions = model.predict(validationRDD.map(_.features)).collect() + assert(predictions.count(_ == 0.0) == predictions.length) + + // Low threshold makes all the predictions 1.0 + model.setThreshold(-10000.0) + predictions = model.predict(validationRDD.map(_.features)).collect() + assert(predictions.count(_ == 1.0) == predictions.length) + } + test("SVM using local random SGD") { val nPoints = 10000 From 5c0dafc2c8734a421206a808b73be67b66264dd7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 13 May 2014 18:32:32 -0700 Subject: [PATCH 337/641] [SPARK-1816] LiveListenerBus dies if a listener throws an exception The solution is to wrap a try / catch / log around the posting of each event to each listener. Author: Andrew Or Closes #759 from andrewor14/listener-die and squashes the following commits: aee5107 [Andrew Or] Merge branch 'master' of github.com:apache/spark into listener-die 370939f [Andrew Or] Remove two layers of indirection 422d278 [Andrew Or] Explicitly throw an exception instead of 1 / 0 0df0e2a [Andrew Or] Try/catch and log exceptions when posting events --- .../spark/scheduler/LiveListenerBus.scala | 36 ++++++++++--- .../spark/scheduler/SparkListenerBus.scala | 50 +++++++++++++------ .../scala/org/apache/spark/util/Utils.scala | 2 +- .../spark/scheduler/SparkListenerSuite.scala | 50 +++++++++++++++++-- 4 files changed, 109 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index dec3316bf7745..36a6e6338faa6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import java.util.concurrent.{LinkedBlockingQueue, Semaphore} import org.apache.spark.Logging +import org.apache.spark.util.Utils /** * Asynchronously passes SparkListenerEvents to registered SparkListeners. @@ -42,7 +43,7 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { private val listenerThread = new Thread("SparkListenerBus") { setDaemon(true) - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { while (true) { eventLock.acquire() // Atomically remove and process this event @@ -77,11 +78,8 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { val eventAdded = eventQueue.offer(event) if (eventAdded) { eventLock.release() - } else if (!queueFullErrorMessageLogged) { - logError("Dropping SparkListenerEvent because no remaining room in event queue. " + - "This likely means one of the SparkListeners is too slow and cannot keep up with the " + - "rate at which tasks are being started by the scheduler.") - queueFullErrorMessageLogged = true + } else { + logQueueFullErrorMessage() } } @@ -96,13 +94,18 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { if (System.currentTimeMillis > finishTime) { return false } - /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify - * add overhead in the general case. */ + /* Sleep rather than using wait/notify, because this is used only for testing and + * wait/notify add overhead in the general case. */ Thread.sleep(10) } true } + /** + * For testing only. Return whether the listener daemon thread is still alive. + */ + def listenerThreadIsAlive: Boolean = synchronized { listenerThread.isAlive } + /** * Return whether the event queue is empty. * @@ -111,6 +114,23 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { */ def queueIsEmpty: Boolean = synchronized { eventQueue.isEmpty } + /** + * Log an error message to indicate that the event queue is full. Do this only once. + */ + private def logQueueFullErrorMessage(): Unit = { + if (!queueFullErrorMessageLogged) { + if (listenerThread.isAlive) { + logError("Dropping SparkListenerEvent because no remaining room in event queue. " + + "This likely means one of the SparkListeners is too slow and cannot keep up with" + + "the rate at which tasks are being started by the scheduler.") + } else { + logError("SparkListenerBus thread is dead! This means SparkListenerEvents have not" + + "been (and will no longer be) propagated to listeners for some time.") + } + queueFullErrorMessageLogged = true + } + } + def stop() { if (!started) { throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 0286aac8769b2..ed9fb24bc8ce8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -20,10 +20,13 @@ package org.apache.spark.scheduler import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import org.apache.spark.Logging +import org.apache.spark.util.Utils + /** * A SparkListenerEvent bus that relays events to its listeners */ -private[spark] trait SparkListenerBus { +private[spark] trait SparkListenerBus extends Logging { // SparkListeners attached to this event bus protected val sparkListeners = new ArrayBuffer[SparkListener] @@ -34,38 +37,53 @@ private[spark] trait SparkListenerBus { } /** - * Post an event to all attached listeners. This does nothing if the event is - * SparkListenerShutdown. + * Post an event to all attached listeners. + * This does nothing if the event is SparkListenerShutdown. */ def postToAll(event: SparkListenerEvent) { event match { case stageSubmitted: SparkListenerStageSubmitted => - sparkListeners.foreach(_.onStageSubmitted(stageSubmitted)) + foreachListener(_.onStageSubmitted(stageSubmitted)) case stageCompleted: SparkListenerStageCompleted => - sparkListeners.foreach(_.onStageCompleted(stageCompleted)) + foreachListener(_.onStageCompleted(stageCompleted)) case jobStart: SparkListenerJobStart => - sparkListeners.foreach(_.onJobStart(jobStart)) + foreachListener(_.onJobStart(jobStart)) case jobEnd: SparkListenerJobEnd => - sparkListeners.foreach(_.onJobEnd(jobEnd)) + foreachListener(_.onJobEnd(jobEnd)) case taskStart: SparkListenerTaskStart => - sparkListeners.foreach(_.onTaskStart(taskStart)) + foreachListener(_.onTaskStart(taskStart)) case taskGettingResult: SparkListenerTaskGettingResult => - sparkListeners.foreach(_.onTaskGettingResult(taskGettingResult)) + foreachListener(_.onTaskGettingResult(taskGettingResult)) case taskEnd: SparkListenerTaskEnd => - sparkListeners.foreach(_.onTaskEnd(taskEnd)) + foreachListener(_.onTaskEnd(taskEnd)) case environmentUpdate: SparkListenerEnvironmentUpdate => - sparkListeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) + foreachListener(_.onEnvironmentUpdate(environmentUpdate)) case blockManagerAdded: SparkListenerBlockManagerAdded => - sparkListeners.foreach(_.onBlockManagerAdded(blockManagerAdded)) + foreachListener(_.onBlockManagerAdded(blockManagerAdded)) case blockManagerRemoved: SparkListenerBlockManagerRemoved => - sparkListeners.foreach(_.onBlockManagerRemoved(blockManagerRemoved)) + foreachListener(_.onBlockManagerRemoved(blockManagerRemoved)) case unpersistRDD: SparkListenerUnpersistRDD => - sparkListeners.foreach(_.onUnpersistRDD(unpersistRDD)) + foreachListener(_.onUnpersistRDD(unpersistRDD)) case applicationStart: SparkListenerApplicationStart => - sparkListeners.foreach(_.onApplicationStart(applicationStart)) + foreachListener(_.onApplicationStart(applicationStart)) case applicationEnd: SparkListenerApplicationEnd => - sparkListeners.foreach(_.onApplicationEnd(applicationEnd)) + foreachListener(_.onApplicationEnd(applicationEnd)) case SparkListenerShutdown => } } + + /** + * Apply the given function to all attached listeners, catching and logging any exception. + */ + private def foreachListener(f: SparkListener => Unit): Unit = { + sparkListeners.foreach { listener => + try { + f(listener) + } catch { + case e: Exception => + logError(s"Listener ${Utils.getFormattedClassName(listener)} threw an exception", e) + } + } + } + } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 0631e54237923..99ef6dd1fa6e3 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1128,7 +1128,7 @@ private[spark] object Utils extends Logging { } /** - * Executes the given block, printing and re-throwing any uncaught exceptions. + * Execute the given block, logging and re-throwing any uncaught exception. * This is particularly useful for wrapping code that runs in a thread, to ensure * that exceptions are printed, and to avoid having to catch Throwable. */ diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 4e9fd07e68a21..5426e578a9ddd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -331,16 +331,47 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } } - def checkNonZeroAvg(m: Traversable[Long], msg: String) { + test("SparkListener moves on if a listener throws an exception") { + val badListener = new BadListener + val jobCounter1 = new BasicJobCounter + val jobCounter2 = new BasicJobCounter + val bus = new LiveListenerBus + + // Propagate events to bad listener first + bus.addListener(badListener) + bus.addListener(jobCounter1) + bus.addListener(jobCounter2) + bus.start() + + // Post events to all listeners, and wait until the queue is drained + (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } + assert(bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + + // The exception should be caught, and the event should be propagated to other listeners + assert(bus.listenerThreadIsAlive) + assert(jobCounter1.count === 5) + assert(jobCounter2.count === 5) + } + + /** + * Assert that the given list of numbers has an average that is greater than zero. + */ + private def checkNonZeroAvg(m: Traversable[Long], msg: String) { assert(m.sum / m.size.toDouble > 0.0, msg) } - class BasicJobCounter extends SparkListener { + /** + * A simple listener that counts the number of jobs observed. + */ + private class BasicJobCounter extends SparkListener { var count = 0 override def onJobEnd(job: SparkListenerJobEnd) = count += 1 } - class SaveStageAndTaskInfo extends SparkListener { + /** + * A simple listener that saves all task infos and task metrics. + */ + private class SaveStageAndTaskInfo extends SparkListener { val stageInfos = mutable.Map[StageInfo, Seq[(TaskInfo, TaskMetrics)]]() var taskInfoMetrics = mutable.Buffer[(TaskInfo, TaskMetrics)]() @@ -358,7 +389,10 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } } - class SaveTaskEvents extends SparkListener { + /** + * A simple listener that saves the task indices for all task events. + */ + private class SaveTaskEvents extends SparkListener { val startedTasks = new mutable.HashSet[Int]() val startedGettingResultTasks = new mutable.HashSet[Int]() val endedTasks = new mutable.HashSet[Int]() @@ -377,4 +411,12 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc startedGettingResultTasks += taskGettingResult.taskInfo.index } } + + /** + * A simple listener that throws an exception on job end. + */ + private class BadListener extends SparkListener { + override def onJobEnd(jobEnd: SparkListenerJobEnd) = { throw new Exception } + } + } From 753b04dea4b04ba9d0dd0011f00e9d70367e76fc Mon Sep 17 00:00:00 2001 From: Ye Xianjin Date: Tue, 13 May 2014 19:03:51 -0700 Subject: [PATCH 338/641] [SPARK-1527] change rootDir*.getName to rootDir*.getAbsolutePath JIRA issue: [SPARK-1527](https://issues.apache.org/jira/browse/SPARK-1527) getName() only gets the last component of the file path. When deleting test-generated directories, we should pass the generated directory's absolute path to DiskBlockManager. Author: Ye Xianjin This patch had conflicts when merged, resolved by Committer: Patrick Wendell Closes #436 from advancedxy/SPARK-1527 and squashes the following commits: 4678bab [Ye Xianjin] change rootDir*.getname to rootDir*.getAbsolutePath so the temporary directories are deleted when the test is finished. --- .../scala/org/apache/spark/storage/DiskBlockManagerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 2167718fd2428..aaa7714049732 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -52,7 +52,7 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before rootDir0.deleteOnExit() rootDir1 = Files.createTempDir() rootDir1.deleteOnExit() - rootDirs = rootDir0.getName + "," + rootDir1.getName + rootDirs = rootDir0.getAbsolutePath + "," + rootDir1.getAbsolutePath println("Created root dirs: " + rootDirs) } From 44233865cf8020741d862d33cc660c88e9315dea Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 13 May 2014 21:23:51 -0700 Subject: [PATCH 339/641] [SQL] Make it possible to create Java/Python SQLContexts from an existing Scala SQLContext. Author: Michael Armbrust Closes #761 from marmbrus/existingContext and squashes the following commits: 4651051 [Michael Armbrust] Make it possible to create Java/Python SQLContexts from an existing Scala SQLContext. --- python/pyspark/sql.py | 7 +++++-- .../org/apache/spark/sql/api/java/JavaSQLContext.scala | 4 ++-- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 6789d7002b3b7..bbe69e7d8f89b 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -28,7 +28,7 @@ class SQLContext: register L{SchemaRDD}s as tables, execute sql over tables, cache tables, and read parquet files. """ - def __init__(self, sparkContext): + def __init__(self, sparkContext, sqlContext = None): """ Create a new SQLContext. @@ -58,10 +58,13 @@ def __init__(self, sparkContext): self._jvm = self._sc._jvm self._pythonToJavaMap = self._jvm.PythonRDD.pythonToJavaMap + if sqlContext: + self._scala_SQLContext = sqlContext + @property def _ssql_ctx(self): """ - Accessor for the JVM SparkSQL context. Subclasses can overrite this property to provide + Accessor for the JVM SparkSQL context. Subclasses can override this property to provide their own JVM Contexts. """ if not hasattr(self, '_scala_SQLContext'): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 57facbe10fc96..6f7d431b9a819 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -33,9 +33,9 @@ import org.apache.spark.util.Utils /** * The entry point for executing Spark SQL queries from a Java program. */ -class JavaSQLContext(sparkContext: JavaSparkContext) { +class JavaSQLContext(val sqlContext: SQLContext) { - val sqlContext = new SQLContext(sparkContext.sc) + def this(sparkContext: JavaSparkContext) = this(new SQLContext(sparkContext.sc)) /** * Executes a query expressed in SQL, returning the result as a JavaSchemaRDD From 92cebada09a7e5a00ab48bcb350a9462949c33eb Mon Sep 17 00:00:00 2001 From: Syed Hashmi Date: Tue, 13 May 2014 21:24:23 -0700 Subject: [PATCH 340/641] [SPARK-1784] Add a new partitioner to allow specifying # of keys per partition This change adds a new partitioner which allows users to specify # of keys per partition. Author: Syed Hashmi Closes #721 from syedhashmi/master and squashes the following commits: 4ca94cc [Syed Hashmi] [SPARK-1784] Add a new partitioner --- .../scala/org/apache/spark/Partitioner.scala | 61 +++++++++++++++++++ .../org/apache/spark/PartitioningSuite.scala | 34 +++++++++++ 2 files changed, 95 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 9155159cf6aeb..62747960618a9 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -156,3 +156,64 @@ class RangePartitioner[K : Ordering : ClassTag, V]( false } } + +/** + * A [[org.apache.spark.Partitioner]] that partitions records into specified bounds + * Default value is 1000. Once all partitions have bounds elements, the partitioner + * allocates 1 element per partition so eventually the smaller partitions are at most + * off by 1 key compared to the larger partitions. + */ +class BoundaryPartitioner[K : Ordering : ClassTag, V]( + partitions: Int, + @transient rdd: RDD[_ <: Product2[K,V]], + private val boundary: Int = 1000) + extends Partitioner { + + // this array keeps track of keys assigned to a partition + // counts[0] refers to # of keys in partition 0 and so on + private val counts: Array[Int] = { + new Array[Int](numPartitions) + } + + def numPartitions = math.abs(partitions) + + /* + * Ideally, this should've been calculated based on # partitions and total keys + * But we are not calling count on RDD here to avoid calling an action. + * User has the flexibility of calling count and passing in any appropriate boundary + */ + def keysPerPartition = boundary + + var currPartition = 0 + + /* + * Pick current partition for the key until we hit the bound for keys / partition, + * start allocating to next partition at that time. + * + * NOTE: In case where we have lets say 2000 keys and user says 3 partitions with 500 + * passed in as boundary, the first 500 will goto P1, 501-1000 go to P2, 1001-1500 go to P3, + * after that, next keys go to one partition at a time. So 1501 goes to P1, 1502 goes to P2, + * 1503 goes to P3 and so on. + */ + def getPartition(key: Any): Int = { + val partition = currPartition + counts(partition) = counts(partition) + 1 + /* + * Since we are filling up a partition before moving to next one (this helps in maintaining + * order of keys, in certain cases, it is possible to end up with empty partitions, like + * 3 partitions, 500 keys / partition and if rdd has 700 keys, 1 partition will be entirely + * empty. + */ + if(counts(currPartition) >= keysPerPartition) + currPartition = (currPartition + 1) % numPartitions + partition + } + + override def equals(other: Any): Boolean = other match { + case r: BoundaryPartitioner[_,_] => + (r.counts.sameElements(counts) && r.boundary == boundary + && r.currPartition == currPartition) + case _ => + false + } +} diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 7c30626a0c421..7d40395803f02 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -66,6 +66,40 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet assert(descendingP4 != p4) } + test("BoundaryPartitioner equality") { + // Make an RDD where all the elements are the same so that the partition range bounds + // are deterministically all the same. + val rdd = sc.parallelize(1.to(4000)).map(x => (x, x)) + + val p2 = new BoundaryPartitioner(2, rdd, 1000) + val p4 = new BoundaryPartitioner(4, rdd, 1000) + val anotherP4 = new BoundaryPartitioner(4, rdd) + + assert(p2 === p2) + assert(p4 === p4) + assert(p2 != p4) + assert(p4 != p2) + assert(p4 === anotherP4) + assert(anotherP4 === p4) + } + + test("BoundaryPartitioner getPartition") { + val rdd = sc.parallelize(1.to(2000)).map(x => (x, x)) + val partitioner = new BoundaryPartitioner(4, rdd, 500) + 1.to(2000).map { element => { + val partition = partitioner.getPartition(element) + if (element <= 500) { + assert(partition === 0) + } else if (element > 501 && element <= 1000) { + assert(partition === 1) + } else if (element > 1001 && element <= 1500) { + assert(partition === 2) + } else if (element > 1501 && element <= 2000) { + assert(partition === 3) + } + }} + } + test("RangePartitioner getPartition") { val rdd = sc.parallelize(1.to(2000)).map(x => (x, x)) // We have different behaviour of getPartition for partitions with less than 1000 and more than From c33b8dcbf65a3a0c5ee5e65cd1dcdbc7da36aa5f Mon Sep 17 00:00:00 2001 From: larvaboy Date: Tue, 13 May 2014 21:26:08 -0700 Subject: [PATCH 341/641] Implement ApproximateCountDistinct for SparkSql Add the implementation for ApproximateCountDistinct to SparkSql. We use the HyperLogLog algorithm implemented in stream-lib, and do the count in two phases: 1) counting the number of distinct elements in each partitions, and 2) merge the HyperLogLog results from different partitions. A simple serializer and test cases are added as well. Author: larvaboy Closes #737 from larvaboy/master and squashes the following commits: bd8ef3f [larvaboy] Add support of user-provided standard deviation to ApproxCountDistinct. 9ba8360 [larvaboy] Fix alignment and null handling issues. 95b4067 [larvaboy] Add a test case for count distinct and approximate count distinct. f57917d [larvaboy] Add the parser for the approximate count. a2d5d10 [larvaboy] Add ApproximateCountDistinct aggregates and functions. 7ad273a [larvaboy] Add SparkSql serializer for HyperLogLog. 1d9aacf [larvaboy] Fix a minor typo in the toString method of the Count case class. 653542b [larvaboy] Fix a couple of minor typos. --- .../apache/spark/rdd/PairRDDFunctions.scala | 6 +- .../apache/spark/sql/catalyst/SqlParser.scala | 7 ++ .../sql/catalyst/expressions/aggregates.scala | 78 ++++++++++++++++++- .../sql/execution/SparkSqlSerializer.scala | 17 ++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 21 ++++- 5 files changed, 122 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 5efb4388f6c71..bc6d204434ad8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -217,7 +217,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Return approximate number of distinct values for each key in this RDD. * The accuracy of approximation can be controlled through the relative standard deviation * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. Uses the provided + * more accurate counts but increase the memory footprint and vice versa. Uses the provided * Partitioner to partition the output RDD. */ def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)] = { @@ -232,7 +232,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Return approximate number of distinct values for each key in this RDD. * The accuracy of approximation can be controlled through the relative standard deviation * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. HashPartitions the + * more accurate counts but increase the memory footprint and vice versa. HashPartitions the * output RDD into numPartitions. * */ @@ -244,7 +244,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Return approximate number of distinct values for each key this RDD. * The accuracy of approximation can be controlled through the relative standard deviation * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. The default value of + * more accurate counts but increase the memory footprint and vice versa. The default value of * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism * level. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index b3a3a1ef1b5eb..f2b9b2c1a3ad5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -93,6 +93,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val AND = Keyword("AND") protected val AS = Keyword("AS") protected val ASC = Keyword("ASC") + protected val APPROXIMATE = Keyword("APPROXIMATE") protected val AVG = Keyword("AVG") protected val BY = Keyword("BY") protected val CAST = Keyword("CAST") @@ -318,6 +319,12 @@ class SqlParser extends StandardTokenParsers with PackratParsers { COUNT ~> "(" ~ "*" <~ ")" ^^ { case _ => Count(Literal(1)) } | COUNT ~> "(" ~ expression <~ ")" ^^ { case dist ~ exp => Count(exp) } | COUNT ~> "(" ~> DISTINCT ~> expression <~ ")" ^^ { case exp => CountDistinct(exp :: Nil) } | + APPROXIMATE ~> COUNT ~> "(" ~> DISTINCT ~> expression <~ ")" ^^ { + case exp => ApproxCountDistinct(exp) + } | + APPROXIMATE ~> "(" ~> floatLit ~ ")" ~ COUNT ~ "(" ~ DISTINCT ~ expression <~ ")" ^^ { + case s ~ _ ~ _ ~ _ ~ _ ~ e => ApproxCountDistinct(e, s.toDouble) + } | FIRST ~> "(" ~> expression <~ ")" ^^ { case exp => First(exp) } | AVG ~> "(" ~> expression <~ ")" ^^ { case exp => Average(exp) } | MIN ~> "(" ~> expression <~ ")" ^^ { case exp => Min(exp) } | diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 7777d372903e2..5dbaaa3b0ce35 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import com.clearspring.analytics.stream.cardinality.HyperLogLog + import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors.TreeNodeException @@ -146,7 +148,6 @@ case class MaxFunction(expr: Expression, base: AggregateExpression) extends Aggr override def eval(input: Row): Any = currentMax } - case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { override def references = child.references override def nullable = false @@ -166,10 +167,47 @@ case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpressi override def references = expressions.flatMap(_.references).toSet override def nullable = false override def dataType = IntegerType - override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")}})" + override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")})" override def newInstance() = new CountDistinctFunction(expressions, this) } +case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double) + extends AggregateExpression with trees.UnaryNode[Expression] { + override def references = child.references + override def nullable = false + override def dataType = child.dataType + override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" + override def newInstance() = new ApproxCountDistinctPartitionFunction(child, this, relativeSD) +} + +case class ApproxCountDistinctMerge(child: Expression, relativeSD: Double) + extends AggregateExpression with trees.UnaryNode[Expression] { + override def references = child.references + override def nullable = false + override def dataType = IntegerType + override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" + override def newInstance() = new ApproxCountDistinctMergeFunction(child, this, relativeSD) +} + +case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05) + extends PartialAggregate with trees.UnaryNode[Expression] { + override def references = child.references + override def nullable = false + override def dataType = IntegerType + override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" + + override def asPartial: SplitEvaluation = { + val partialCount = + Alias(ApproxCountDistinctPartition(child, relativeSD), "PartialApproxCountDistinct")() + + SplitEvaluation( + ApproxCountDistinctMerge(partialCount.toAttribute, relativeSD), + partialCount :: Nil) + } + + override def newInstance() = new CountDistinctFunction(child :: Nil, this) +} + case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { override def references = child.references override def nullable = false @@ -269,6 +307,42 @@ case class CountFunction(expr: Expression, base: AggregateExpression) extends Ag override def eval(input: Row): Any = count } +case class ApproxCountDistinctPartitionFunction( + expr: Expression, + base: AggregateExpression, + relativeSD: Double) + extends AggregateFunction { + def this() = this(null, null, 0) // Required for serialization. + + private val hyperLogLog = new HyperLogLog(relativeSD) + + override def update(input: Row): Unit = { + val evaluatedExpr = expr.eval(input) + if (evaluatedExpr != null) { + hyperLogLog.offer(evaluatedExpr) + } + } + + override def eval(input: Row): Any = hyperLogLog +} + +case class ApproxCountDistinctMergeFunction( + expr: Expression, + base: AggregateExpression, + relativeSD: Double) + extends AggregateFunction { + def this() = this(null, null, 0) // Required for serialization. + + private val hyperLogLog = new HyperLogLog(relativeSD) + + override def update(input: Row): Unit = { + val evaluatedExpr = expr.eval(input) + hyperLogLog.addAll(evaluatedExpr.asInstanceOf[HyperLogLog]) + } + + override def eval(input: Row): Any = hyperLogLog.cardinality() +} + case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 1c6e29b3cdee9..94c2a249ef8f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -21,6 +21,7 @@ import java.nio.ByteBuffer import scala.reflect.ClassTag +import com.clearspring.analytics.stream.cardinality.HyperLogLog import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Serializer, Kryo} @@ -44,6 +45,8 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co kryo.register(classOf[scala.collection.Map[_,_]], new MapSerializer) kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericRow]) kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericMutableRow]) + kryo.register(classOf[com.clearspring.analytics.stream.cardinality.HyperLogLog], + new HyperLogLogSerializer) kryo.register(classOf[scala.collection.mutable.ArrayBuffer[_]]) kryo.register(classOf[scala.math.BigDecimal], new BigDecimalSerializer) kryo.setReferences(false) @@ -81,6 +84,20 @@ private[sql] class BigDecimalSerializer extends Serializer[BigDecimal] { } } +private[sql] class HyperLogLogSerializer extends Serializer[HyperLogLog] { + def write(kryo: Kryo, output: Output, hyperLogLog: HyperLogLog) { + val bytes = hyperLogLog.getBytes() + output.writeInt(bytes.length) + output.writeBytes(bytes) + } + + def read(kryo: Kryo, input: Input, tpe: Class[HyperLogLog]): HyperLogLog = { + val length = input.readInt() + val bytes = input.readBytes(length) + HyperLogLog.Builder.build(bytes) + } +} + /** * Maps do not have a no arg constructor and so cannot be serialized by default. So, we serialize * them as `Array[(k,v)]`. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index e966d89c30cf5..524549eb544fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -96,8 +96,25 @@ class SQLQuerySuite extends QueryTest { test("count") { checkAnswer( sql("SELECT COUNT(*) FROM testData2"), - testData2.count() - ) + testData2.count()) + } + + test("count distinct") { + checkAnswer( + sql("SELECT COUNT(DISTINCT b) FROM testData2"), + 2) + } + + test("approximate count distinct") { + checkAnswer( + sql("SELECT APPROXIMATE COUNT(DISTINCT a) FROM testData2"), + 3) + } + + test("approximate count distinct with user provided standard deviation") { + checkAnswer( + sql("SELECT APPROXIMATE(0.04) COUNT(DISTINCT a) FROM testData2"), + 3) } // No support for primitive nulls yet. From 7bb9a521f35eb19576c6cc2da3fd385910270e46 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 13 May 2014 23:24:51 -0700 Subject: [PATCH 342/641] Revert "[SPARK-1784] Add a new partitioner to allow specifying # of keys per partition" This reverts commit 92cebada09a7e5a00ab48bcb350a9462949c33eb. --- .../scala/org/apache/spark/Partitioner.scala | 61 ------------------- .../org/apache/spark/PartitioningSuite.scala | 34 ----------- 2 files changed, 95 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 62747960618a9..9155159cf6aeb 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -156,64 +156,3 @@ class RangePartitioner[K : Ordering : ClassTag, V]( false } } - -/** - * A [[org.apache.spark.Partitioner]] that partitions records into specified bounds - * Default value is 1000. Once all partitions have bounds elements, the partitioner - * allocates 1 element per partition so eventually the smaller partitions are at most - * off by 1 key compared to the larger partitions. - */ -class BoundaryPartitioner[K : Ordering : ClassTag, V]( - partitions: Int, - @transient rdd: RDD[_ <: Product2[K,V]], - private val boundary: Int = 1000) - extends Partitioner { - - // this array keeps track of keys assigned to a partition - // counts[0] refers to # of keys in partition 0 and so on - private val counts: Array[Int] = { - new Array[Int](numPartitions) - } - - def numPartitions = math.abs(partitions) - - /* - * Ideally, this should've been calculated based on # partitions and total keys - * But we are not calling count on RDD here to avoid calling an action. - * User has the flexibility of calling count and passing in any appropriate boundary - */ - def keysPerPartition = boundary - - var currPartition = 0 - - /* - * Pick current partition for the key until we hit the bound for keys / partition, - * start allocating to next partition at that time. - * - * NOTE: In case where we have lets say 2000 keys and user says 3 partitions with 500 - * passed in as boundary, the first 500 will goto P1, 501-1000 go to P2, 1001-1500 go to P3, - * after that, next keys go to one partition at a time. So 1501 goes to P1, 1502 goes to P2, - * 1503 goes to P3 and so on. - */ - def getPartition(key: Any): Int = { - val partition = currPartition - counts(partition) = counts(partition) + 1 - /* - * Since we are filling up a partition before moving to next one (this helps in maintaining - * order of keys, in certain cases, it is possible to end up with empty partitions, like - * 3 partitions, 500 keys / partition and if rdd has 700 keys, 1 partition will be entirely - * empty. - */ - if(counts(currPartition) >= keysPerPartition) - currPartition = (currPartition + 1) % numPartitions - partition - } - - override def equals(other: Any): Boolean = other match { - case r: BoundaryPartitioner[_,_] => - (r.counts.sameElements(counts) && r.boundary == boundary - && r.currPartition == currPartition) - case _ => - false - } -} diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 7d40395803f02..7c30626a0c421 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -66,40 +66,6 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet assert(descendingP4 != p4) } - test("BoundaryPartitioner equality") { - // Make an RDD where all the elements are the same so that the partition range bounds - // are deterministically all the same. - val rdd = sc.parallelize(1.to(4000)).map(x => (x, x)) - - val p2 = new BoundaryPartitioner(2, rdd, 1000) - val p4 = new BoundaryPartitioner(4, rdd, 1000) - val anotherP4 = new BoundaryPartitioner(4, rdd) - - assert(p2 === p2) - assert(p4 === p4) - assert(p2 != p4) - assert(p4 != p2) - assert(p4 === anotherP4) - assert(anotherP4 === p4) - } - - test("BoundaryPartitioner getPartition") { - val rdd = sc.parallelize(1.to(2000)).map(x => (x, x)) - val partitioner = new BoundaryPartitioner(4, rdd, 500) - 1.to(2000).map { element => { - val partition = partitioner.getPartition(element) - if (element <= 500) { - assert(partition === 0) - } else if (element > 501 && element <= 1000) { - assert(partition === 1) - } else if (element > 1001 && element <= 1500) { - assert(partition === 2) - } else if (element > 1501 && element <= 2000) { - assert(partition === 3) - } - }} - } - test("RangePartitioner getPartition") { val rdd = sc.parallelize(1.to(2000)).map(x => (x, x)) // We have different behaviour of getPartition for partitions with less than 1000 and more than From 6ce0884446d3571fd6e9d967a080a59c657543b1 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 13 May 2014 23:27:22 -0700 Subject: [PATCH 343/641] [SQL] Improve column pruning. Fixed a bug that was preventing us from ever pruning beneath Joins. ## TPC-DS Q3 ### Before: ``` Aggregate false, [d_year#12,i_brand#65,i_brand_id#64], [d_year#12,i_brand_id#64 AS brand_id#0,i_brand#65 AS brand#1,SUM(PartialSum#79) AS sum_agg#2] Exchange (HashPartitioning [d_year#12:0,i_brand#65:1,i_brand_id#64:2], 150) Aggregate true, [d_year#12,i_brand#65,i_brand_id#64], [d_year#12,i_brand#65,i_brand_id#64,SUM(CAST(ss_ext_sales_price#49, DoubleType)) AS PartialSum#79] Project [d_year#12:6,i_brand#65:59,i_brand_id#64:58,ss_ext_sales_price#49:43] HashJoin [ss_item_sk#36], [i_item_sk#57], BuildRight Exchange (HashPartitioning [ss_item_sk#36:30], 150) HashJoin [d_date_sk#6], [ss_sold_date_sk#34], BuildRight Exchange (HashPartitioning [d_date_sk#6:0], 150) Filter (d_moy#14:8 = 12) HiveTableScan [d_date_sk#6,d_date_id#7,d_date#8,d_month_seq#9,d_week_seq#10,d_quarter_seq#11,d_year#12,d_dow#13,d_moy#14,d_dom#15,d_qoy#16,d_fy_year#17,d_fy_quarter_seq#18,d_fy_week_seq#19,d_day_name#20,d_quarter_name#21,d_holiday#22,d_weekend#23,d_following_holiday#24,d_first_dom#25,d_last_dom#26,d_same_day_ly#27,d_same_day_lq#28,d_current_day#29,d_current_week#30,d_current_month#31,d_current_quarter#32,d_current_year#33], (MetastoreRelation default, date_dim, Some(dt)), None Exchange (HashPartitioning [ss_sold_date_sk#34:0], 150) HiveTableScan [ss_sold_date_sk#34,ss_sold_time_sk#35,ss_item_sk#36,ss_customer_sk#37,ss_cdemo_sk#38,ss_hdemo_sk#39,ss_addr_sk#40,ss_store_sk#41,ss_promo_sk#42,ss_ticket_number#43,ss_quantity#44,ss_wholesale_cost#45,ss_list_price#46,ss_sales_price#47,ss_ext_discount_amt#48,ss_ext_sales_price#49,ss_ext_wholesale_cost#50,ss_ext_list_price#51,ss_ext_tax#52,ss_coupon_amt#53,ss_net_paid#54,ss_net_paid_inc_tax#55,ss_net_profit#56], (MetastoreRelation default, store_sales, None), None Exchange (HashPartitioning [i_item_sk#57:0], 150) Filter (i_manufact_id#70:13 = 436) HiveTableScan [i_item_sk#57,i_item_id#58,i_rec_start_date#59,i_rec_end_date#60,i_item_desc#61,i_current_price#62,i_wholesale_cost#63,i_brand_id#64,i_brand#65,i_class_id#66,i_class#67,i_category_id#68,i_category#69,i_manufact_id#70,i_manufact#71,i_size#72,i_formulation#73,i_color#74,i_units#75,i_container#76,i_manager_id#77,i_product_name#78], (MetastoreRelation default, item, None), None ``` ### After ``` Aggregate false, [d_year#172,i_brand#225,i_brand_id#224], [d_year#172,i_brand_id#224 AS brand_id#160,i_brand#225 AS brand#161,SUM(PartialSum#239) AS sum_agg#162] Exchange (HashPartitioning [d_year#172:0,i_brand#225:1,i_brand_id#224:2], 150) Aggregate true, [d_year#172,i_brand#225,i_brand_id#224], [d_year#172,i_brand#225,i_brand_id#224,SUM(CAST(ss_ext_sales_price#209, DoubleType)) AS PartialSum#239] Project [d_year#172:1,i_brand#225:5,i_brand_id#224:3,ss_ext_sales_price#209:0] HashJoin [ss_item_sk#196], [i_item_sk#217], BuildRight Exchange (HashPartitioning [ss_item_sk#196:2], 150) Project [ss_ext_sales_price#209:2,d_year#172:1,ss_item_sk#196:3] HashJoin [d_date_sk#166], [ss_sold_date_sk#194], BuildRight Exchange (HashPartitioning [d_date_sk#166:0], 150) Project [d_date_sk#166:0,d_year#172:1] Filter (d_moy#174:2 = 12) HiveTableScan [d_date_sk#166,d_year#172,d_moy#174], (MetastoreRelation default, date_dim, Some(dt)), None Exchange (HashPartitioning [ss_sold_date_sk#194:2], 150) HiveTableScan [ss_ext_sales_price#209,ss_item_sk#196,ss_sold_date_sk#194], (MetastoreRelation default, store_sales, None), None Exchange (HashPartitioning [i_item_sk#217:1], 150) Project [i_brand_id#224:0,i_item_sk#217:1,i_brand#225:2] Filter (i_manufact_id#230:3 = 436) HiveTableScan [i_brand_id#224,i_item_sk#217,i_brand#225,i_manufact_id#230], (MetastoreRelation default, item, None), None ``` Author: Michael Armbrust Closes #729 from marmbrus/fixPruning and squashes the following commits: 5feeff0 [Michael Armbrust] Improve column pruning. --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 3037d45cc6e35..406ffd6801e98 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -25,13 +25,13 @@ import org.apache.spark.sql.catalyst.types._ object Optimizer extends RuleExecutor[LogicalPlan] { val batches = - Batch("ConstantFolding", Once, + Batch("ConstantFolding", FixedPoint(100), NullPropagation, ConstantFolding, BooleanSimplification, SimplifyFilters, SimplifyCasts) :: - Batch("Filter Pushdown", Once, + Batch("Filter Pushdown", FixedPoint(100), CombineFilters, PushPredicateThroughProject, PushPredicateThroughInnerJoin, @@ -49,17 +49,19 @@ object Optimizer extends RuleExecutor[LogicalPlan] { */ object ColumnPruning extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // Eliminate attributes that are not needed to calculate the specified aggregates. case a @ Aggregate(_, _, child) if (child.outputSet -- a.references).nonEmpty => - // Project away references that are not needed to calculate the required aggregates. a.copy(child = Project(a.references.toSeq, child)) + // Eliminate unneeded attributes from either side of a Join. case Project(projectList, Join(left, right, joinType, condition)) => // Collect the list of off references required either above or to evaluate the condition. val allReferences: Set[Attribute] = projectList.flatMap(_.references).toSet ++ condition.map(_.references).getOrElse(Set.empty) - /** Applies a projection when the child is producing unnecessary attributes */ + + /** Applies a projection only when the child is producing unnecessary attributes */ def prunedChild(c: LogicalPlan) = - if ((allReferences.filter(c.outputSet.contains) -- c.outputSet).nonEmpty) { + if ((c.outputSet -- allReferences.filter(c.outputSet.contains)).nonEmpty) { Project(allReferences.filter(c.outputSet.contains).toSeq, c) } else { c @@ -67,6 +69,7 @@ object ColumnPruning extends Rule[LogicalPlan] { Project(projectList, Join(prunedChild(left), prunedChild(right), joinType, condition)) + // Combine adjacent Projects. case Project(projectList1, Project(projectList2, child)) => // Create a map of Aliases to their values from the child projection. // e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c -> Alias(a + b, c)). @@ -83,6 +86,9 @@ object ColumnPruning extends Rule[LogicalPlan] { }).asInstanceOf[Seq[NamedExpression]] Project(substitutedProjection, child) + + // Eliminate no-op Projects + case Project(projectList, child) if(child.output == projectList) => child } } From b22952fa1f21c0b93208846b5e1941a9d2578c6f Mon Sep 17 00:00:00 2001 From: Koert Kuipers Date: Wed, 14 May 2014 00:10:12 -0700 Subject: [PATCH 344/641] SPARK-1801. expose InterruptibleIterator and TaskKilledException in deve... ...loper api Author: Koert Kuipers Closes #764 from koertkuipers/feat-rdd-developerapi and squashes the following commits: 8516dd2 [Koert Kuipers] SPARK-1801. expose InterruptibleIterator and TaskKilledException in developer api --- .../scala/org/apache/spark/InterruptibleIterator.scala | 6 +++++- .../main/scala/org/apache/spark/TaskKilledException.scala | 8 ++++++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala index ec11dbbffaaf8..f40baa8e43592 100644 --- a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala +++ b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala @@ -17,11 +17,15 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * An iterator that wraps around an existing iterator to provide task killing functionality. * It works by checking the interrupted flag in [[TaskContext]]. */ -private[spark] class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T]) +@DeveloperApi +class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T]) extends Iterator[T] { def hasNext: Boolean = { diff --git a/core/src/main/scala/org/apache/spark/TaskKilledException.scala b/core/src/main/scala/org/apache/spark/TaskKilledException.scala index cbd6b2866e4f9..ad487c4efb87a 100644 --- a/core/src/main/scala/org/apache/spark/TaskKilledException.scala +++ b/core/src/main/scala/org/apache/spark/TaskKilledException.scala @@ -17,7 +17,11 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi + /** - * Exception for a task getting killed. + * :: DeveloperApi :: + * Exception thrown when a task is explicitly killed (i.e., task failure is expected). */ -private[spark] class TaskKilledException extends RuntimeException +@DeveloperApi +class TaskKilledException extends RuntimeException From 54ae8328bd7d052ba347768cfb02cb5dfdd8045e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 14 May 2014 00:37:57 -0700 Subject: [PATCH 345/641] Fix dep exclusion: avro-ipc, not avro, depends on netty. Author: Marcelo Vanzin Closes #763 from vanzin/netty-dep-hell and squashes the following commits: dfb6ce2 [Marcelo Vanzin] Fix dep exclusion: avro-ipc, not avro, depends on netty. --- pom.xml | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index 4d4c5f63e44a3..786b6d4984d86 100644 --- a/pom.xml +++ b/pom.xml @@ -496,18 +496,16 @@ org.apache.avro avro ${avro.version} - - - io.netty - netty - -
      org.apache.avro avro-ipc ${avro.version} + + io.netty + netty + org.mortbay.jetty jetty From 69f750228f3ec8537a93da08e712596fa8004143 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 14 May 2014 00:54:33 -0700 Subject: [PATCH 346/641] [SPARK-1769] Executor loss causes NPE race condition This PR replaces the Schedulable data structures in Pool.scala with thread-safe ones from java. Note that Scala's `with SynchronizedBuffer` trait is soon to be deprecated in 2.11 because it is ["inherently unreliable"](http://www.scala-lang.org/api/2.11.0/index.html#scala.collection.mutable.SynchronizedBuffer). We should slowly drift away from `SynchronizedBuffer` in other places too. Note that this PR introduces an API-breaking change; `sc.getAllPools` now returns an Array rather than an ArrayBuffer. This is because we want this method to return an immutable copy rather than one may potentially confuse the user if they try to modify the copy, which takes no effect on the original data structure. Author: Andrew Or Closes #762 from andrewor14/pool-npe and squashes the following commits: 383e739 [Andrew Or] JavaConverters -> JavaConversions 3f32981 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pool-npe 769be19 [Andrew Or] Assorted minor changes 2189247 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pool-npe 05ad9e9 [Andrew Or] Fix test - contains is not the same as containsKey 0921ea0 [Andrew Or] var -> val 07d720c [Andrew Or] Synchronize Schedulable data structures --- .../scala/org/apache/spark/SparkContext.scala | 20 +++++++----- .../org/apache/spark/scheduler/Pool.scala | 31 ++++++++++--------- .../apache/spark/scheduler/Schedulable.scala | 6 ++-- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../scheduler/TaskSchedulerImplSuite.scala | 2 +- 5 files changed, 35 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c43b4fd6d926f..032b3d744c619 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -17,15 +17,17 @@ package org.apache.spark +import scala.language.implicitConversions + import java.io._ import java.net.URI import java.util.concurrent.atomic.AtomicInteger import java.util.{Properties, UUID} import java.util.UUID.randomUUID import scala.collection.{Map, Set} +import scala.collection.JavaConversions._ import scala.collection.generic.Growable -import scala.collection.mutable.{ArrayBuffer, HashMap} -import scala.language.implicitConversions +import scala.collection.mutable.HashMap import scala.reflect.{ClassTag, classTag} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -836,18 +838,22 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * Return pools for fair scheduler - * TODO(xiajunluan): We should take nested pools into account + * :: DeveloperApi :: + * Return pools for fair scheduler */ - def getAllPools: ArrayBuffer[Schedulable] = { - taskScheduler.rootPool.schedulableQueue + @DeveloperApi + def getAllPools: Seq[Schedulable] = { + // TODO(xiajunluan): We should take nested pools into account + taskScheduler.rootPool.schedulableQueue.toSeq } /** + * :: DeveloperApi :: * Return the pool associated with the given name, if one exists */ + @DeveloperApi def getPoolForName(pool: String): Option[Schedulable] = { - taskScheduler.rootPool.schedulableNameToSchedulable.get(pool) + Option(taskScheduler.rootPool.schedulableNameToSchedulable.get(pool)) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 187672c4e19e7..174b73221afc0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -17,8 +17,10 @@ package org.apache.spark.scheduler +import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} + +import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap import org.apache.spark.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode @@ -35,18 +37,15 @@ private[spark] class Pool( extends Schedulable with Logging { - var schedulableQueue = new ArrayBuffer[Schedulable] - var schedulableNameToSchedulable = new HashMap[String, Schedulable] - + val schedulableQueue = new ConcurrentLinkedQueue[Schedulable] + val schedulableNameToSchedulable = new ConcurrentHashMap[String, Schedulable] var weight = initWeight var minShare = initMinShare var runningTasks = 0 - var priority = 0 // A pool's stage id is used to break the tie in scheduling. var stageId = -1 - var name = poolName var parent: Pool = null @@ -60,19 +59,20 @@ private[spark] class Pool( } override def addSchedulable(schedulable: Schedulable) { - schedulableQueue += schedulable - schedulableNameToSchedulable(schedulable.name) = schedulable + require(schedulable != null) + schedulableQueue.add(schedulable) + schedulableNameToSchedulable.put(schedulable.name, schedulable) schedulable.parent = this } override def removeSchedulable(schedulable: Schedulable) { - schedulableQueue -= schedulable - schedulableNameToSchedulable -= schedulable.name + schedulableQueue.remove(schedulable) + schedulableNameToSchedulable.remove(schedulable.name) } override def getSchedulableByName(schedulableName: String): Schedulable = { - if (schedulableNameToSchedulable.contains(schedulableName)) { - return schedulableNameToSchedulable(schedulableName) + if (schedulableNameToSchedulable.containsKey(schedulableName)) { + return schedulableNameToSchedulable.get(schedulableName) } for (schedulable <- schedulableQueue) { val sched = schedulable.getSchedulableByName(schedulableName) @@ -95,11 +95,12 @@ private[spark] class Pool( shouldRevive } - override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + override def getSortedTaskSetQueue: ArrayBuffer[TaskSetManager] = { var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] - val sortedSchedulableQueue = schedulableQueue.sortWith(taskSetSchedulingAlgorithm.comparator) + val sortedSchedulableQueue = + schedulableQueue.toSeq.sortWith(taskSetSchedulingAlgorithm.comparator) for (schedulable <- sortedSchedulableQueue) { - sortedTaskSetQueue ++= schedulable.getSortedTaskSetQueue() + sortedTaskSetQueue ++= schedulable.getSortedTaskSetQueue } sortedTaskSetQueue } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala index ed24eb6a549dd..a87ef030e69c2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import java.util.concurrent.ConcurrentLinkedQueue + import scala.collection.mutable.ArrayBuffer import org.apache.spark.scheduler.SchedulingMode.SchedulingMode @@ -28,7 +30,7 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode private[spark] trait Schedulable { var parent: Pool // child queues - def schedulableQueue: ArrayBuffer[Schedulable] + def schedulableQueue: ConcurrentLinkedQueue[Schedulable] def schedulingMode: SchedulingMode def weight: Int def minShare: Int @@ -42,5 +44,5 @@ private[spark] trait Schedulable { def getSchedulableByName(name: String): Schedulable def executorLost(executorId: String, host: String): Unit def checkSpeculatableTasks(): Boolean - def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] + def getSortedTaskSetQueue: ArrayBuffer[TaskSetManager] } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 5a68f38bc5844..ffd1d9432682b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -222,7 +222,7 @@ private[spark] class TaskSchedulerImpl( // Build a list of tasks to assign to each worker. val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = shuffledOffers.map(o => o.cores).toArray - val sortedTaskSets = rootPool.getSortedTaskSetQueue() + val sortedTaskSets = rootPool.getSortedTaskSetQueue for (taskSet <- sortedTaskSets) { logDebug("parentName: %s, name: %s, runningTasks: %s".format( taskSet.parent.name, taskSet.name, taskSet.runningTasks)) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index a8b605c5b212a..7532da88c6065 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -117,7 +117,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin } def resourceOffer(rootPool: Pool): Int = { - val taskSetQueue = rootPool.getSortedTaskSetQueue() + val taskSetQueue = rootPool.getSortedTaskSetQueue /* Just for Test*/ for (manager <- taskSetQueue) { logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format( From 68f28dabe9c7679be82e684385be216319beb610 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 14 May 2014 04:17:32 -0700 Subject: [PATCH 347/641] Fixed streaming examples docs to use run-example instead of spark-submit Pretty self-explanatory Author: Tathagata Das Closes #722 from tdas/example-fix and squashes the following commits: 7839979 [Tathagata Das] Minor changes. 0673441 [Tathagata Das] Fixed java docs of java streaming example e687123 [Tathagata Das] Fixed scala style errors. 9b8d112 [Tathagata Das] Fixed streaming examples docs to use run-example instead of spark-submit. --- .../streaming/JavaCustomReceiver.java | 13 ++--- .../streaming/JavaFlumeEventCount.java | 6 +- .../streaming/JavaKafkaWordCount.java | 6 +- .../streaming/JavaNetworkWordCount.java | 13 +++-- .../examples/streaming/ActorWordCount.scala | 6 +- .../examples/streaming/CustomReceiver.scala | 19 +++--- .../examples/streaming/FlumeEventCount.scala | 9 ++- .../examples/streaming/HdfsWordCount.scala | 5 +- .../examples/streaming/KafkaWordCount.scala | 6 +- .../examples/streaming/MQTTWordCount.scala | 10 ++-- .../examples/streaming/NetworkWordCount.scala | 14 +++-- .../RecoverableNetworkWordCount.scala | 7 +-- .../streaming/StatefulNetworkWordCount.scala | 6 +- .../streaming/TwitterPopularTags.scala | 22 ++++++- .../examples/streaming/ZeroMQWordCount.scala | 8 +-- .../clickstream/PageViewGenerator.scala | 10 ++-- .../clickstream/PageViewStream.scala | 7 ++- .../twitter/TwitterInputDStream.scala | 58 +++++++++++-------- 18 files changed, 130 insertions(+), 95 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java index 7f558f3ee713a..5622df5ce03ff 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java @@ -19,6 +19,7 @@ import com.google.common.collect.Lists; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; @@ -48,25 +49,23 @@ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run org.apache.spark.examples.streaming.JavaCustomReceiver local[2] localhost 9999` + * `$ bin/run-example org.apache.spark.examples.streaming.JavaCustomReceiver localhost 9999` */ public class JavaCustomReceiver extends Receiver { private static final Pattern SPACE = Pattern.compile(" "); public static void main(String[] args) { - if (args.length < 3) { - System.err.println("Usage: JavaNetworkWordCount \n" + - "In local mode, should be 'local[n]' with n > 1"); + if (args.length < 2) { + System.err.println("Usage: JavaNetworkWordCount "); System.exit(1); } StreamingExamples.setStreamingLogLevels(); // Create the context with a 1 second batch size - JavaStreamingContext ssc = new JavaStreamingContext(args[0], "JavaNetworkWordCount", - new Duration(1000), System.getenv("SPARK_HOME"), - JavaStreamingContext.jarOfClass(JavaNetworkWordCount.class)); + SparkConf sparkConf = new SparkConf().setAppName("JavaCustomReceiver"); + JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, new Duration(1000)); // Create a input stream with the custom receiver on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java index 400b68c2215b3..da56637fe891a 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java @@ -33,10 +33,12 @@ * Your Flume AvroSink should be pointed to this address. * * Usage: JavaFlumeEventCount - * * is the host the Flume receiver will be started on - a receiver * creates a server and listens for flume events. * is the port the Flume receiver will listen on. + * + * To run this example: + * `$ bin/run-example org.apache.spark.examples.streaming.JavaFlumeEventCount ` */ public final class JavaFlumeEventCount { private JavaFlumeEventCount() { @@ -56,7 +58,7 @@ public static void main(String[] args) { Duration batchInterval = new Duration(2000); SparkConf sparkConf = new SparkConf().setAppName("JavaFlumeEventCount"); JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, batchInterval); - JavaReceiverInputDStream flumeStream = FlumeUtils.createStream(ssc, "localhost", port); + JavaReceiverInputDStream flumeStream = FlumeUtils.createStream(ssc, host, port); flumeStream.count(); diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java index 6a74cc50d19ed..16ae9a3319ee2 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java @@ -40,15 +40,15 @@ /** * Consumes messages from one or more topics in Kafka and does wordcount. + * * Usage: JavaKafkaWordCount * is a list of one or more zookeeper servers that make quorum * is the name of kafka consumer group * is a list of one or more kafka topics to consume from * is the number of threads the kafka consumer should use * - * Example: - * `./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.JavaKafkaWordCount zoo01,zoo02, \ + * To run this example: + * `$ bin/run-example org.apache.spark.examples.streaming.JavaKafkaWordCount zoo01,zoo02, \ * zoo03 my-consumer-group topic1,topic2 1` */ diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java index e5cbd39f437c2..45bcedebb4117 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java @@ -24,7 +24,7 @@ import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.examples.streaming.StreamingExamples; +import org.apache.spark.api.java.StorageLevels; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; @@ -41,8 +41,7 @@ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.JavaNetworkWordCount localhost 9999` + * `$ bin/run-example org.apache.spark.examples.streaming.JavaNetworkWordCount localhost 9999` */ public final class JavaNetworkWordCount { private static final Pattern SPACE = Pattern.compile(" "); @@ -54,13 +53,17 @@ public static void main(String[] args) { } StreamingExamples.setStreamingLogLevels(); - SparkConf sparkConf = new SparkConf().setAppName("JavaNetworkWordCount"); + // Create the context with a 1 second batch size + SparkConf sparkConf = new SparkConf().setAppName("JavaNetworkWordCount"); JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, new Duration(1000)); // Create a JavaReceiverInputDStream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') - JavaReceiverInputDStream lines = ssc.socketTextStream(args[0], Integer.parseInt(args[1])); + // Note that no duplication in storage level only for running locally. + // Replication necessary in distributed scenario for fault tolerance. + JavaReceiverInputDStream lines = ssc.socketTextStream( + args[0], Integer.parseInt(args[1]), StorageLevels.MEMORY_AND_DISK_SER); JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala index e29e16a9c1b17..b433082dce1a2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala @@ -130,11 +130,9 @@ object FeederActor { * and describe the AkkaSystem that Spark Sample feeder is running on. * * To run this example locally, you may run Feeder Actor as - * `./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.FeederActor 127.0.1.1 9999` + * `$ bin/run-example org.apache.spark.examples.streaming.FeederActor 127.0.1.1 9999` * and then run the example - * `./bin/spark-submit examples.jar --class org.apache.spark.examples.streaming.ActorWordCount \ - * 127.0.1.1 9999` + * `$ bin/run-example org.apache.spark.examples.streaming.ActorWordCount 127.0.1.1 9999` */ object ActorWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala index e317e2d36ae43..6bb659fbd8be8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.streaming import java.io.{InputStreamReader, BufferedReader, InputStream} import java.net.Socket -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ @@ -30,32 +30,27 @@ import org.apache.spark.streaming.receiver.Receiver * Custom Receiver that receives data over a socket. Received bytes is interpreted as * text and \n delimited lines are considered as records. They are then counted and printed. * - * Usage: CustomReceiver - * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. - * and of the TCP server that Spark Streaming would connect to receive data. - * * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run org.apache.spark.examples.streaming.CustomReceiver local[2] localhost 9999` + * `$ bin/run-example org.apache.spark.examples.streaming.CustomReceiver localhost 9999` */ object CustomReceiver { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: NetworkWordCount \n" + - "In local mode, should be 'local[n]' with n > 1") + if (args.length < 2) { + System.err.println("Usage: CustomReceiver ") System.exit(1) } StreamingExamples.setStreamingLogLevels() // Create the context with a 1 second batch size - val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + val sparkConf = new SparkConf().setAppName("CustomReceiver") + val ssc = new StreamingContext(sparkConf, Seconds(1)) // Create a input stream with the custom receiver on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') - val lines = ssc.receiverStream(new CustomReceiver(args(1), args(2).toInt)) + val lines = ssc.receiverStream(new CustomReceiver(args(0), args(1).toInt)) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.print() diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala index 38362edac27f8..20e7df7c45b1b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala @@ -31,14 +31,16 @@ import org.apache.spark.util.IntParam * Your Flume AvroSink should be pointed to this address. * * Usage: FlumeEventCount - * * is the host the Flume receiver will be started on - a receiver * creates a server and listens for flume events. * is the port the Flume receiver will listen on. + * + * To run this example: + * `$ bin/run-example org.apache.spark.examples.streaming.FlumeEventCount ` */ object FlumeEventCount { def main(args: Array[String]) { - if (args.length != 3) { + if (args.length < 2) { System.err.println( "Usage: FlumeEventCount ") System.exit(1) @@ -49,8 +51,9 @@ object FlumeEventCount { val Array(host, IntParam(port)) = args val batchInterval = Milliseconds(2000) - val sparkConf = new SparkConf().setAppName("FlumeEventCount") + // Create the context and set the batch size + val sparkConf = new SparkConf().setAppName("FlumeEventCount") val ssc = new StreamingContext(sparkConf, batchInterval) // Create a flume stream diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala index 55ac48cfb6d10..6c24bc3ad09e0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala @@ -27,8 +27,9 @@ import org.apache.spark.streaming.StreamingContext._ * is the directory that Spark Streaming will use to find and read new text files. * * To run this on your local machine on directory `localdir`, run this example - * `$ ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.HdfsWordCount localdir` + * $ bin/run-example \ + * org.apache.spark.examples.streaming.HdfsWordCount localdir + * * Then create a text file in `localdir` and the words in the file will get counted. */ object HdfsWordCount { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index 3af806981f37a..566ba6f911e02 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -35,9 +35,9 @@ import org.apache.spark.SparkConf * is the number of threads the kafka consumer should use * * Example: - * `./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.KafkaWordCount local[2] zoo01,zoo02,zoo03 \ - * my-consumer-group topic1,topic2 1` + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.KafkaWordCount zoo01,zoo02,zoo03 \ + * my-consumer-group topic1,topic2 1` */ object KafkaWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala index 3a10daa9ab84a..e4283e04a1b11 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala @@ -75,14 +75,14 @@ object MQTTPublisher { * Example Java code for Mqtt Publisher and Subscriber can be found here * https://bitbucket.org/mkjinesh/mqttclient * Usage: MQTTWordCount -\ * and describe where Mqtt publisher is running. + * and describe where Mqtt publisher is running. * * To run this example locally, you may run publisher as - * `$ ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.MQTTPublisher tcp://localhost:1883 foo` + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.MQTTPublisher tcp://localhost:1883 foo` * and run the example as - * `$ ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.MQTTWordCount tcp://localhost:1883 foo` + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.MQTTWordCount tcp://localhost:1883 foo` */ object MQTTWordCount { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala index ad7a199b2c0ab..ae0a08c6cdb1a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala @@ -23,7 +23,7 @@ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.storage.StorageLevel /** - * Counts words in text encoded with UTF8 received from the network every second. + * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. * * Usage: NetworkWordCount * and describe the TCP server that Spark Streaming would connect to receive data. @@ -31,8 +31,7 @@ import org.apache.spark.storage.StorageLevel * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.NetworkWordCount localhost 9999` + * `$ bin/run-example org.apache.spark.examples.streaming.NetworkWordCount localhost 9999` */ object NetworkWordCount { def main(args: Array[String]) { @@ -42,13 +41,16 @@ object NetworkWordCount { } StreamingExamples.setStreamingLogLevels() - val sparkConf = new SparkConf().setAppName("NetworkWordCount"); + // Create the context with a 1 second batch size + val sparkConf = new SparkConf().setAppName("NetworkWordCount") val ssc = new StreamingContext(sparkConf, Seconds(1)) - // Create a NetworkInputDStream on target ip:port and count the + // Create a socket stream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') - val lines = ssc.socketTextStream(args(0), args(1).toInt, StorageLevel.MEMORY_ONLY_SER) + // Note that no duplication in storage level only for running locally. + // Replication necessary in distributed scenario for fault tolerance. + val lines = ssc.socketTextStream(args(0), args(1).toInt, StorageLevel.MEMORY_AND_DISK_SER) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.print() diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index ace785d9fe4c5..6af3a0f33efc2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -46,8 +46,7 @@ import org.apache.spark.util.IntParam * * and run the example as * - * `$ ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.RecoverableNetworkWordCount \ + * `$ ./bin/run-example org.apache.spark.examples.streaming.RecoverableNetworkWordCount \ * localhost 9999 ~/checkpoint/ ~/out` * * If the directory ~/checkpoint/ does not exist (e.g. running for the first time), it will create @@ -57,7 +56,7 @@ import org.apache.spark.util.IntParam * * To run this example in a local standalone cluster with automatic driver recovery, * - * `$ ./spark-class org.apache.spark.deploy.Client -s launch \ + * `$ bin/spark-class org.apache.spark.deploy.Client -s launch \ * \ * org.apache.spark.examples.streaming.RecoverableNetworkWordCount \ * localhost 9999 ~/checkpoint ~/out` @@ -81,7 +80,7 @@ object RecoverableNetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(sparkConf, Seconds(1)) - // Create a NetworkInputDStream on target ip:port and count the + // Create a socket stream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') val lines = ssc.socketTextStream(ip, port) val words = lines.flatMap(_.split(" ")) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala index 5e1415f3cc536..daa1ced63c701 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala @@ -31,8 +31,8 @@ import org.apache.spark.streaming.StreamingContext._ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./bin/spark-submit examples.jar - * --class org.apache.spark.examples.streaming.StatefulNetworkWordCount localhost 9999` + * `$ bin/run-example + * org.apache.spark.examples.streaming.StatefulNetworkWordCount localhost 9999` */ object StatefulNetworkWordCount { def main(args: Array[String]) { @@ -51,7 +51,7 @@ object StatefulNetworkWordCount { Some(currentCount + previousCount) } - val sparkConf = new SparkConf().setAppName("NetworkWordCumulativeCountUpdateStateByKey") + val sparkConf = new SparkConf().setAppName("StatefulNetworkWordCount") // Create the context with a 1 second batch size val ssc = new StreamingContext(sparkConf, Seconds(1)) ssc.checkpoint(".") diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala index 1ddff22cb8a42..f55d23ab3924b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala @@ -28,13 +28,29 @@ import org.apache.spark.SparkConf * stream. The stream is instantiated with credentials and optionally filters supplied by the * command line arguments. * + * Run this on your local machine as + * */ object TwitterPopularTags { def main(args: Array[String]) { + if (args.length < 4) { + System.err.println("Usage: TwitterPopularTags " + + " []") + System.exit(1) + } StreamingExamples.setStreamingLogLevels() - val filters = args + val Array(consumerKey, consumerSecret, accessToken, accessTokenSecret) = args.take(4) + val filters = args.takeRight(args.length - 4) + + // Set the system properties so that Twitter4j library used by twitter stream + // can use them to generat OAuth credentials + System.setProperty("twitter4j.oauth.consumerKey", consumerKey) + System.setProperty("twitter4j.oauth.consumerSecret", consumerSecret) + System.setProperty("twitter4j.oauth.accessToken", accessToken) + System.setProperty("twitter4j.oauth.accessTokenSecret", accessTokenSecret) + val sparkConf = new SparkConf().setAppName("TwitterPopularTags") val ssc = new StreamingContext(sparkConf, Seconds(2)) val stream = TwitterUtils.createStream(ssc, None, filters) @@ -52,13 +68,13 @@ object TwitterPopularTags { // Print popular hashtags topCounts60.foreachRDD(rdd => { - val topList = rdd.take(5) + val topList = rdd.take(10) println("\nPopular topics in last 60 seconds (%s total):".format(rdd.count())) topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))} }) topCounts10.foreachRDD(rdd => { - val topList = rdd.take(5) + val topList = rdd.take(10) println("\nPopular topics in last 10 seconds (%s total):".format(rdd.count())) topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))} }) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala index 7ade3f1018ee8..79905af381a12 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala @@ -68,11 +68,11 @@ object SimpleZeroMQPublisher { * and describe where zeroMq publisher is running. * * To run this example locally, you may run publisher as - * `$ ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` * and run the example as - * `$ ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.ZeroMQWordCount tcp://127.0.1.1:1234 foo` + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.ZeroMQWordCount tcp://127.0.1.1:1234 foo` */ // scalastyle:on object ZeroMQWordCount { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala index 97e0cb92078dc..8402491b62671 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala @@ -40,11 +40,13 @@ object PageView extends Serializable { /** Generates streaming events to simulate page views on a website. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10 - * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444 * - * When running this, you may want to set the root logging level to ERROR in - * conf/log4j.properties to reduce the verbosity of the output. + * To run the generator + * `$ bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10` + * To process the generated stream + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444` + * */ // scalastyle:on object PageViewGenerator { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala index d30ceffbe29cb..d9b886eff77cc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala @@ -26,8 +26,11 @@ import org.apache.spark.examples.streaming.StreamingExamples * operators available in Spark streaming. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10 - * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + * To run the generator + * `$ bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10` + * To process the generated stream + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444` */ // scalastyle:on object PageViewStream { diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala index 7bca1407116fa..5ea2e5549d7df 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala @@ -63,36 +63,48 @@ class TwitterReceiver( storageLevel: StorageLevel ) extends Receiver[Status](storageLevel) with Logging { - var twitterStream: TwitterStream = _ + private var twitterStream: TwitterStream = _ def onStart() { - twitterStream = new TwitterStreamFactory().getInstance(twitterAuth) - twitterStream.addListener(new StatusListener { - def onStatus(status: Status) = { - store(status) - } - // Unimplemented - def onDeletionNotice(statusDeletionNotice: StatusDeletionNotice) {} - def onTrackLimitationNotice(i: Int) {} - def onScrubGeo(l: Long, l1: Long) {} - def onStallWarning(stallWarning: StallWarning) {} - def onException(e: Exception) { - restart("Error receiving tweets", e) - } - }) + try { + val newTwitterStream = new TwitterStreamFactory().getInstance(twitterAuth) + newTwitterStream.addListener(new StatusListener { + def onStatus(status: Status) = { + store(status) + } + // Unimplemented + def onDeletionNotice(statusDeletionNotice: StatusDeletionNotice) {} + def onTrackLimitationNotice(i: Int) {} + def onScrubGeo(l: Long, l1: Long) {} + def onStallWarning(stallWarning: StallWarning) {} + def onException(e: Exception) { + restart("Error receiving tweets", e) + } + }) - val query = new FilterQuery - if (filters.size > 0) { - query.track(filters.toArray) - twitterStream.filter(query) - } else { - twitterStream.sample() + val query = new FilterQuery + if (filters.size > 0) { + query.track(filters.toArray) + newTwitterStream.filter(query) + } else { + newTwitterStream.sample() + } + setTwitterStream(newTwitterStream) + logInfo("Twitter receiver started") + } catch { + case e: Exception => restart("Error starting Twitter stream", e) } - logInfo("Twitter receiver started") } def onStop() { - twitterStream.shutdown() + setTwitterStream(null) logInfo("Twitter receiver stopped") } + + private def setTwitterStream(newTwitterStream: TwitterStream) = synchronized { + if (twitterStream != null) { + twitterStream.shutdown() + } + twitterStream = newTwitterStream + } } From 2e5a7cde223c8bf6d34e46b27ac94a965441584d Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 14 May 2014 09:38:33 -0700 Subject: [PATCH 348/641] SPARK-1827. LICENSE and NOTICE files need a refresh to contain transitive dependency info LICENSE and NOTICE policy is explained here: http://www.apache.org/dev/licensing-howto.html http://www.apache.org/legal/3party.html This leads to the following changes. First, this change enables two extensions to maven-shade-plugin in assembly/ that will try to include and merge all NOTICE and LICENSE files. This can't hurt. This generates a consolidated NOTICE file that I manually added to NOTICE. Next, a list of all dependencies and their licenses was generated: `mvn ... license:aggregate-add-third-party` to create: `target/generated-sources/license/THIRD-PARTY.txt` Each dependency is listed with one or more licenses. Determine the most-compatible license for each if there is more than one. For "unknown" license dependencies, I manually evaluateD their license. Many are actually Apache projects or components of projects covered already. The only non-trivial one was Colt, which has its own (compatible) license. I ignored Apache-licensed and public domain dependencies as these require no further action (beyond NOTICE above). BSD and MIT licenses (permissive Category A licenses) are evidently supposed to be mentioned in LICENSE, so I added a section without output from the THIRD-PARTY.txt file appropriately. Everything else, Category B licenses, are evidently mentioned in NOTICE (?) Same there. LICENSE contained some license statements for source code that is redistributed. I left this as I think that is the right place to put it. Author: Sean Owen Closes #770 from srowen/SPARK-1827 and squashes the following commits: a764504 [Sean Owen] Add LICENSE and NOTICE info for all transitive dependencies as of 1.0 --- LICENSE | 103 +++++++++ NOTICE | 572 ++++++++++++++++++++++++++++++++++++++++++++++- assembly/pom.xml | 2 + 3 files changed, 671 insertions(+), 6 deletions(-) diff --git a/LICENSE b/LICENSE index 1c1c2c0255fa9..383f079df8c8b 100644 --- a/LICENSE +++ b/LICENSE @@ -428,3 +428,106 @@ LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON A THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +======================================================================== +For colt: +======================================================================== + +Copyright (c) 1999 CERN - European Organization for Nuclear Research. +Permission to use, copy, modify, distribute and sell this software and its documentation for any purpose is hereby granted without fee, provided that the above copyright notice appear in all copies and that both that copyright notice and this permission notice appear in supporting documentation. CERN makes no representations about the suitability of this software for any purpose. It is provided "as is" without expressed or implied warranty. + +Packages hep.aida.* + +Written by Pavel Binko, Dino Ferrero Merlino, Wolfgang Hoschek, Tony Johnson, Andreas Pfeiffer, and others. Check the FreeHEP home page for more info. Permission to use and/or redistribute this work is granted under the terms of the LGPL License, with the exception that any usage related to military applications is expressly forbidden. The software and documentation made available under the terms of this license are provided with no warranty. + + +======================================================================== +Fo SnapTree: +======================================================================== + +SNAPTREE LICENSE + +Copyright (c) 2009-2012 Stanford University, unless otherwise specified. +All rights reserved. + +This software was developed by the Pervasive Parallelism Laboratory of +Stanford University, California, USA. + +Permission to use, copy, modify, and distribute this software in source +or binary form for any purpose with or without fee is hereby granted, +provided that the following conditions are met: + + 1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + + 3. Neither the name of Stanford University nor the names of its + contributors may be used to endorse or promote products derived + from this software without specific prior written permission. + + +THIS SOFTWARE IS PROVIDED BY THE REGENTS AND CONTRIBUTORS ``AS IS'' AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE REGENTS OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT +LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY +OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF +SUCH DAMAGE. + + +======================================================================== +BSD-style licenses +======================================================================== + +The following components are provided under a BSD-style license. See project link for details. + + (BSD 3 Clause) core (com.github.fommil.netlib:core:1.1.2 - https://github.com/fommil/netlib-java/core) + (BSD 3-clause style license) jblas (org.jblas:jblas:1.2.3 - http://jblas.org/) + (BSD License) AntLR Parser Generator (antlr:antlr:2.7.7 - http://www.antlr.org/) + (BSD License) Javolution (javolution:javolution:5.5.1 - http://javolution.org) + (BSD licence) ANTLR ST4 4.0.4 (org.antlr:ST4:4.0.4 - http://www.stringtemplate.org) + (BSD licence) ANTLR StringTemplate (org.antlr:stringtemplate:3.2.1 - http://www.stringtemplate.org) + (BSD style) Hamcrest Core (org.hamcrest:hamcrest-core:1.1 - no url defined) + (BSD) JLine (jline:jline:0.9.94 - http://jline.sourceforge.net) + (BSD) ParaNamer Core (com.thoughtworks.paranamer:paranamer:2.3 - http://paranamer.codehaus.org/paranamer) + (BSD) ParaNamer Core (com.thoughtworks.paranamer:paranamer:2.6 - http://paranamer.codehaus.org/paranamer) + (BSD-like) (The BSD License) jline (org.scala-lang:jline:2.10.4 - http://www.scala-lang.org/) + (BSD-like) Scala Actors library (org.scala-lang:scala-actors:2.10.4 - http://www.scala-lang.org/) + (BSD-like) Scala Compiler (org.scala-lang:scala-compiler:2.10.4 - http://www.scala-lang.org/) + (BSD-like) Scala Compiler (org.scala-lang:scala-reflect:2.10.4 - http://www.scala-lang.org/) + (BSD-like) Scala Library (org.scala-lang:scala-library:2.10.4 - http://www.scala-lang.org/) + (BSD-like) Scalap (org.scala-lang:scalap:2.10.4 - http://www.scala-lang.org/) + (BSD-style) scalacheck (org.scalacheck:scalacheck_2.10:1.10.0 - http://www.scalacheck.org) + (BSD-style) spire (org.spire-math:spire_2.10:0.7.1 - http://spire-math.org) + (BSD-style) spire-macros (org.spire-math:spire-macros_2.10:0.7.1 - http://spire-math.org) + (New BSD License) Kryo (com.esotericsoftware.kryo:kryo:2.21 - http://code.google.com/p/kryo/) + (New BSD License) MinLog (com.esotericsoftware.minlog:minlog:1.2 - http://code.google.com/p/minlog/) + (New BSD License) ReflectASM (com.esotericsoftware.reflectasm:reflectasm:1.07 - http://code.google.com/p/reflectasm/) + (New BSD license) Protocol Buffer Java API (com.google.protobuf:protobuf-java:2.5.0 - http://code.google.com/p/protobuf) + (New BSD license) Protocol Buffer Java API (org.spark-project.protobuf:protobuf-java:2.4.1-shaded - http://code.google.com/p/protobuf) + (The BSD License) Fortran to Java ARPACK (net.sourceforge.f2j:arpack_combined_all:0.1 - http://f2j.sourceforge.net) + (The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net) + (The New BSD License) Py4J (net.sf.py4j:py4j:0.8.1 - http://py4j.sourceforge.net/) + (Two-clause BSD-style license) JUnit-Interface (com.novocode:junit-interface:0.10 - http://github.com/szeiger/junit-interface/) + (ISC/BSD License) jbcrypt (org.mindrot:jbcrypt:0.3m - http://www.mindrot.org/) + +======================================================================== +MIT licenses +======================================================================== + +The following components are provided under the MIT License. See project link for details. + + (MIT License) JCL 1.1.1 implemented over SLF4J (org.slf4j:jcl-over-slf4j:1.7.5 - http://www.slf4j.org) + (MIT License) JUL to SLF4J bridge (org.slf4j:jul-to-slf4j:1.7.5 - http://www.slf4j.org) + (MIT License) SLF4J API Module (org.slf4j:slf4j-api:1.7.5 - http://www.slf4j.org) + (MIT License) SLF4J LOG4J-12 Binding (org.slf4j:slf4j-log4j12:1.7.5 - http://www.slf4j.org) + (MIT License) pyrolite (org.spark-project:pyrolite:2.0.1 - http://pythonhosted.org/Pyro4/) + (MIT License) scopt (com.github.scopt:scopt_2.10:3.2.0 - https://github.com/scopt/scopt) + (The MIT License) Mockito (org.mockito:mockito-all:1.8.5 - http://www.mockito.org) diff --git a/NOTICE b/NOTICE index 42f6c3a835725..452aef2871652 100644 --- a/NOTICE +++ b/NOTICE @@ -4,11 +4,571 @@ Copyright 2014 The Apache Software Foundation. This product includes software developed at The Apache Software Foundation (http://www.apache.org/). -In addition, this product includes: -- JUnit (http://www.junit.org) is a testing framework for Java. We included it - under the terms of the Eclipse Public License v1.0. +======================================================================== +Common Development and Distribution License 1.0 +======================================================================== -- JTransforms (https://sites.google.com/site/piotrwendykier/software/jtransforms) - provides fast transforms in Java. It is tri-licensed, and we included it under - the terms of the Mozilla Public License v1.1. +The following components are provided under the Common Development and Distribution License 1.0. See project link for details. + + (CDDL 1.0) Glassfish Jasper (org.mortbay.jetty:jsp-2.1:6.1.14 - http://jetty.mortbay.org/project/modules/jsp-2.1) + (CDDL 1.0) Servlet Specification 2.5 API (org.mortbay.jetty:servlet-api-2.5:6.1.14 - http://jetty.mortbay.org/project/modules/servlet-api-2.5) + (COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0) (GNU General Public Library) Streaming API for XML (javax.xml.stream:stax-api:1.0-2 - no url defined) + (Common Development and Distribution License (CDDL) v1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp) + +======================================================================== +Common Development and Distribution License 1.1 +======================================================================== + +The following components are provided under the Common Development and Distribution License 1.1. See project link for details. + + (CDDL 1.1) (GPL2 w/ CPE) JAXB API bundle for GlassFish V3 (javax.xml.bind:jaxb-api:2.2.2 - https://jaxb.dev.java.net/) + (CDDL 1.1) (GPL2 w/ CPE) JAXB RI (com.sun.xml.bind:jaxb-impl:2.2.3-1 - http://jaxb.java.net/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:1.8 - https://jersey.dev.java.net/jersey-core/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:1.9 - https://jersey.java.net/jersey-core/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-guice (com.sun.jersey.contribs:jersey-guice:1.9 - https://jersey.java.net/jersey-contribs/jersey-guice/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:1.8 - https://jersey.dev.java.net/jersey-json/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:1.9 - https://jersey.java.net/jersey-json/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:1.8 - https://jersey.dev.java.net/jersey-server/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:1.9 - https://jersey.java.net/jersey-server/) + +======================================================================== +Common Public License 1.0 +======================================================================== + +The following components are provided under the Common Public 1.0 License. See project link for details. + + (Common Public License Version 1.0) JUnit (junit:junit-dep:4.10 - http://junit.org) + (Common Public License Version 1.0) JUnit (junit:junit:3.8.1 - http://junit.org) + (Common Public License Version 1.0) JUnit (junit:junit:4.8.2 - http://junit.org) + +======================================================================== +Eclipse Public License 1.0 +======================================================================== + +The following components are provided under the Eclipse Public License 1.0. See project link for details. + + (Eclipse Public License - Version 1.0) mqtt-client (org.eclipse.paho:mqtt-client:0.4.0 - http://www.eclipse.org/paho/mqtt-client) + (Eclipse Public License v1.0) Eclipse JDT Core (org.eclipse.jdt:core:3.1.1 - http://www.eclipse.org/jdt/) + +======================================================================== +Mozilla Public License 1.0 +======================================================================== + +The following components are provided under the Mozilla Public License 1.0. See project link for details. + + (GPL) (LGPL) (MPL) JTransforms (com.github.rwl:jtransforms:2.4.0 - http://sourceforge.net/projects/jtransforms/) + (Mozilla Public License Version 1.1) jamon-runtime (org.jamon:jamon-runtime:2.3.1 - http://www.jamon.org/jamon-runtime/) + + + +======================================================================== +NOTICE files +======================================================================== + +The following NOTICEs are pertain to software distributed with this project. + + +// ------------------------------------------------------------------ +// NOTICE file corresponding to the section 4d of The Apache License, +// Version 2.0, in this case for +// ------------------------------------------------------------------ + +Apache Avro +Copyright 2009-2013 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +Apache Commons Codec +Copyright 2002-2009 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + +-------------------------------------------------------------------------------- +src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java contains +test data from http://aspell.sourceforge.net/test/batch0.tab. + +Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org). Verbatim copying +and distribution of this entire article is permitted in any medium, +provided this notice is preserved. +-------------------------------------------------------------------------------- + +Apache HttpComponents HttpClient +Copyright 1999-2011 The Apache Software Foundation + +This project contains annotations derived from JCIP-ANNOTATIONS +Copyright (c) 2005 Brian Goetz and Tim Peierls. See http://www.jcip.net + +Apache HttpComponents HttpCore +Copyright 2005-2011 The Apache Software Foundation + +Curator Recipes +Copyright 2011-2014 The Apache Software Foundation + +Curator Framework +Copyright 2011-2014 The Apache Software Foundation + +Curator Client +Copyright 2011-2014 The Apache Software Foundation + +Apache Geronimo +Copyright 2003-2008 The Apache Software Foundation + +Activation 1.1 +Copyright 2003-2007 The Apache Software Foundation + +Apache Commons Lang +Copyright 2001-2014 The Apache Software Foundation + +This product includes software from the Spring Framework, +under the Apache License 2.0 (see: StringUtils.containsWhitespace()) + +Apache log4j +Copyright 2007 The Apache Software Foundation + +# Compress LZF + +This library contains efficient implementation of LZF compression format, +as well as additional helper classes that build on JDK-provided gzip (deflat) +codec. + +## Licensing + +Library is licensed under Apache License 2.0, as per accompanying LICENSE file. + +## Credit + +Library has been written by Tatu Saloranta (tatu.saloranta@iki.fi). +It was started at Ning, inc., as an official Open Source process used by +platform backend, but after initial versions has been developed outside of +Ning by supporting community. + +Other contributors include: + +* Jon Hartlaub (first versions of streaming reader/writer; unit tests) +* Cedrik Lime: parallel LZF implementation + +Various community members have contributed bug reports, and suggested minor +fixes; these can be found from file "VERSION.txt" in SCM. + +Objenesis +Copyright 2006-2009 Joe Walnes, Henri Tremblay, Leonardo Mesquita + +Apache Commons Net +Copyright 2001-2010 The Apache Software Foundation + + The Netty Project + ================= + +Please visit the Netty web site for more information: + + * http://netty.io/ + +Copyright 2011 The Netty Project + +The Netty Project licenses this file to you under the Apache License, +version 2.0 (the "License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at: + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, WITHOUT +WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the +License for the specific language governing permissions and limitations +under the License. + +Also, please refer to each LICENSE..txt file, which is located in +the 'license' directory of the distribution file, for the license terms of the +components that this product depends on. + +------------------------------------------------------------------------------- +This product contains the extensions to Java Collections Framework which has +been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: + + * LICENSE: + * license/LICENSE.jsr166y.txt (Public Domain) + * HOMEPAGE: + * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ + * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ + +This product contains a modified version of Robert Harder's Public Domain +Base64 Encoder and Decoder, which can be obtained at: + + * LICENSE: + * license/LICENSE.base64.txt (Public Domain) + * HOMEPAGE: + * http://iharder.sourceforge.net/current/java/base64/ + +This product contains a modified version of 'JZlib', a re-implementation of +zlib in pure Java, which can be obtained at: + + * LICENSE: + * license/LICENSE.jzlib.txt (BSD Style License) + * HOMEPAGE: + * http://www.jcraft.com/jzlib/ + +This product optionally depends on 'Protocol Buffers', Google's data +interchange format, which can be obtained at: + + * LICENSE: + * license/LICENSE.protobuf.txt (New BSD License) + * HOMEPAGE: + * http://code.google.com/p/protobuf/ + +This product optionally depends on 'SLF4J', a simple logging facade for Java, +which can be obtained at: + + * LICENSE: + * license/LICENSE.slf4j.txt (MIT License) + * HOMEPAGE: + * http://www.slf4j.org/ + +This product optionally depends on 'Apache Commons Logging', a logging +framework, which can be obtained at: + + * LICENSE: + * license/LICENSE.commons-logging.txt (Apache License 2.0) + * HOMEPAGE: + * http://commons.apache.org/logging/ + +This product optionally depends on 'Apache Log4J', a logging framework, +which can be obtained at: + + * LICENSE: + * license/LICENSE.log4j.txt (Apache License 2.0) + * HOMEPAGE: + * http://logging.apache.org/log4j/ + +This product optionally depends on 'JBoss Logging', a logging framework, +which can be obtained at: + + * LICENSE: + * license/LICENSE.jboss-logging.txt (GNU LGPL 2.1) + * HOMEPAGE: + * http://anonsvn.jboss.org/repos/common/common-logging-spi/ + +This product optionally depends on 'Apache Felix', an open source OSGi +framework implementation, which can be obtained at: + + * LICENSE: + * license/LICENSE.felix.txt (Apache License 2.0) + * HOMEPAGE: + * http://felix.apache.org/ + +This product optionally depends on 'Webbit', a Java event based +WebSocket and HTTP server: + + * LICENSE: + * license/LICENSE.webbit.txt (BSD License) + * HOMEPAGE: + * https://github.com/joewalnes/webbit + +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +Jackson core and extension components may be licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +mesos +Copyright 2014 The Apache Software Foundation + +Apache Thrift +Copyright 2006-2010 The Apache Software Foundation. + + Apache Ant + Copyright 1999-2013 The Apache Software Foundation + + The task is based on code Copyright (c) 2002, Landmark + Graphics Corp that has been kindly donated to the Apache Software + Foundation. + +Apache Commons IO +Copyright 2002-2012 The Apache Software Foundation + +Apache Commons Math +Copyright 2001-2013 The Apache Software Foundation + +=============================================================================== + +The inverse error function implementation in the Erf class is based on CUDA +code developed by Mike Giles, Oxford-Man Institute of Quantitative Finance, +and published in GPU Computing Gems, volume 2, 2010. +=============================================================================== + +The BracketFinder (package org.apache.commons.math3.optimization.univariate) +and PowellOptimizer (package org.apache.commons.math3.optimization.general) +classes are based on the Python code in module "optimize.py" (version 0.5) +developed by Travis E. Oliphant for the SciPy library (http://www.scipy.org/) +Copyright © 2003-2009 SciPy Developers. +=============================================================================== + +The LinearConstraint, LinearObjectiveFunction, LinearOptimizer, +RelationShip, SimplexSolver and SimplexTableau classes in package +org.apache.commons.math3.optimization.linear include software developed by +Benjamin McCann (http://www.benmccann.com) and distributed with +the following copyright: Copyright 2009 Google Inc. +=============================================================================== + +This product includes software developed by the +University of Chicago, as Operator of Argonne National +Laboratory. +The LevenbergMarquardtOptimizer class in package +org.apache.commons.math3.optimization.general includes software +translated from the lmder, lmpar and qrsolv Fortran routines +from the Minpack package +Minpack Copyright Notice (1999) University of Chicago. All rights reserved +=============================================================================== + +The GraggBulirschStoerIntegrator class in package +org.apache.commons.math3.ode.nonstiff includes software translated +from the odex Fortran routine developed by E. Hairer and G. Wanner. +Original source copyright: +Copyright (c) 2004, Ernst Hairer +=============================================================================== + +The EigenDecompositionImpl class in package +org.apache.commons.math3.linear includes software translated +from some LAPACK Fortran routines. Original source copyright: +Copyright (c) 1992-2008 The University of Tennessee. All rights reserved. +=============================================================================== + +The MersenneTwister class in package org.apache.commons.math3.random +includes software translated from the 2002-01-26 version of +the Mersenne-Twister generator written in C by Makoto Matsumoto and Takuji +Nishimura. Original source copyright: +Copyright (C) 1997 - 2002, Makoto Matsumoto and Takuji Nishimura, +All rights reserved +=============================================================================== + +The LocalizedFormatsTest class in the unit tests is an adapted version of +the OrekitMessagesTest class from the orekit library distributed under the +terms of the Apache 2 licence. Original source copyright: +Copyright 2010 CS Systèmes d'Information +=============================================================================== + +The HermiteInterpolator class and its corresponding test have been imported from +the orekit library distributed under the terms of the Apache 2 licence. Original +source copyright: +Copyright 2010-2012 CS Systèmes d'Information +=============================================================================== + +The creation of the package "o.a.c.m.analysis.integration.gauss" was inspired +by an original code donated by Sébastien Brisard. +=============================================================================== + +The complete text of licenses and disclaimers associated with the the original +sources enumerated above at the time of code translation are in the LICENSE.txt +file. + +This product currently only contains code developed by authors +of specific components, as identified by the source code files; +if such notes are missing files have been created by +Tatu Saloranta. + +For additional credits (generally to people who reported problems) +see CREDITS file. + +Apache Commons Lang +Copyright 2001-2011 The Apache Software Foundation + +Apache Commons Compress +Copyright 2002-2012 The Apache Software Foundation + +Apache Commons CLI +Copyright 2001-2009 The Apache Software Foundation + +Google Guice - Extensions - Servlet +Copyright 2006-2011 Google, Inc. + +Google Guice - Core Library +Copyright 2006-2011 Google, Inc. + +Apache Jakarta HttpClient +Copyright 1999-2007 The Apache Software Foundation + +Apache Hive +Copyright 2008-2013 The Apache Software Foundation + +This product includes software developed by The Apache Software +Foundation (http://www.apache.org/). + +This product includes software developed by The JDBM Project +(http://jdbm.sourceforge.net/). + +This product includes/uses ANTLR (http://www.antlr.org/), +Copyright (c) 2003-2011, Terrence Parr. + +This product includes/uses StringTemplate (http://www.stringtemplate.org/), +Copyright (c) 2011, Terrence Parr. + +This product includes/uses ASM (http://asm.ow2.org/), +Copyright (c) 2000-2007 INRIA, France Telecom. + +This product includes/uses org.json (http://www.json.org/java/index.html), +Copyright (c) 2002 JSON.org + +This product includes/uses JLine (http://jline.sourceforge.net/), +Copyright (c) 2002-2006, Marc Prud'hommeaux . + +This product includes/uses SQLLine (http://sqlline.sourceforge.net), +Copyright (c) 2002, 2003, 2004, 2005 Marc Prud'hommeaux . + +This product includes/uses SLF4J (http://www.slf4j.org/), +Copyright (c) 2004-2010 QOS.ch + +This product includes/uses Bootstrap (http://twitter.github.com/bootstrap/), +Copyright (c) 2012 Twitter, Inc. + +This product includes/uses Glyphicons (http://glyphicons.com/), +Copyright (c) 2010 - 2012 Jan Kovarík + +This product includes DataNucleus (http://www.datanucleus.org/) +Copyright 2008-2008 DataNucleus + +This product includes Guava (http://code.google.com/p/guava-libraries/) +Copyright (C) 2006 Google Inc. + +This product includes JavaEWAH (http://code.google.com/p/javaewah/) +Copyright (C) 2011 Google Inc. + +Apache Commons Pool +Copyright 1999-2009 The Apache Software Foundation + +========================================================================= +== NOTICE file corresponding to section 4(d) of the Apache License, == +== Version 2.0, in this case for the DataNucleus distribution. == +========================================================================= + +=================================================================== +This product includes software developed by many individuals, +including the following: +=================================================================== +Erik Bengtson +Andy Jefferson + +=================================================================== +This product has included contributions from some individuals, +including the following: +=================================================================== + +=================================================================== +This product has included contributions from some individuals, +including the following: +=================================================================== +Joerg von Frantzius +Thomas Marti +Barry Haddow +Marco Schulze +Ralph Ullrich +David Ezzio +Brendan de Beer +David Eaves +Martin Taal +Tony Lai +Roland Szabo +Marcus Mennemeier +Xuan Baldauf +Eric Sultan + +=================================================================== +This product also includes software developed by the TJDO project +(http://tjdo.sourceforge.net/). +=================================================================== + +=================================================================== +This product includes software developed by many individuals, +including the following: +=================================================================== +Andy Jefferson +Erik Bengtson +Joerg von Frantzius +Marco Schulze + +=================================================================== +This product has included contributions from some individuals, +including the following: +=================================================================== +Barry Haddow +Ralph Ullrich +David Ezzio +Brendan de Beer +David Eaves +Martin Taal +Tony Lai +Roland Szabo +Anton Troshin (Timesten) + +=================================================================== +This product also includes software developed by the Apache Commons project +(http://commons.apache.org/). +=================================================================== + +Apache Java Data Objects (JDO) +Copyright 2005-2006 The Apache Software Foundation + +========================================================================= +== NOTICE file corresponding to section 4(d) of the Apache License, == +== Version 2.0, in this case for the Apache Derby distribution. == +========================================================================= + +Apache Derby +Copyright 2004-2008 The Apache Software Foundation + +Portions of Derby were originally developed by +International Business Machines Corporation and are +licensed to the Apache Software Foundation under the +"Software Grant and Corporate Contribution License Agreement", +informally known as the "Derby CLA". +The following copyright notice(s) were affixed to portions of the code +with which this file is now or was at one time distributed +and are placed here unaltered. + +(C) Copyright 1997,2004 International Business Machines Corporation. All rights reserved. + +(C) Copyright IBM Corp. 2003. + +The portion of the functionTests under 'nist' was originally +developed by the National Institute of Standards and Technology (NIST), +an agency of the United States Department of Commerce, and adapted by +International Business Machines Corporation in accordance with the NIST +Software Acknowledgment and Redistribution document at +http://www.itl.nist.gov/div897/ctg/sql_form.htm + +Apache Commons Collections +Copyright 2001-2008 The Apache Software Foundation + +Apache Commons Configuration +Copyright 2001-2008 The Apache Software Foundation + +Apache Jakarta Commons Digester +Copyright 2001-2006 The Apache Software Foundation + +Apache Commons BeanUtils +Copyright 2000-2008 The Apache Software Foundation + +Apache Avro Mapred API +Copyright 2009-2013 The Apache Software Foundation + +Apache Avro IPC +Copyright 2009-2013 The Apache Software Foundation diff --git a/assembly/pom.xml b/assembly/pom.xml index 6c4d46aeb67bd..abd8935339992 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -122,6 +122,8 @@ log4j.properties + + From d1d41ccee49a5c093cb61c791c01f64f2076b83e Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Wed, 14 May 2014 09:45:33 -0700 Subject: [PATCH 349/641] SPARK-1818 Freshen Mesos documentation Place more emphasis on using precompiled binary versions of Spark and Mesos instead of encouraging the reader to compile from source. Author: Andrew Ash Closes #756 from ash211/spark-1818 and squashes the following commits: 7ef3b33 [Andrew Ash] Brief explanation of the interactions between Spark and Mesos e7dea8e [Andrew Ash] Add troubleshooting and debugging section 956362d [Andrew Ash] Don't need to pass spark.executor.uri into the spark shell de3353b [Andrew Ash] Wrap to 100char 7ebf6ef [Andrew Ash] Polish on the section on Mesos Master URLs 3dcc2c1 [Andrew Ash] Use --tgz parameter of make-distribution 41b68ed [Andrew Ash] Period at end of sentence; formatting on :5050 8bf2c53 [Andrew Ash] Update site.MESOS_VERSIOn to match /pom.xml 74f2040 [Andrew Ash] SPARK-1818 Freshen Mesos documentation --- docs/_config.yml | 2 +- docs/running-on-mesos.md | 200 +++++++++++++++++++++++++++++++++------ 2 files changed, 174 insertions(+), 28 deletions(-) diff --git a/docs/_config.yml b/docs/_config.yml index d177e38f884ca..45b78fe724a50 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -7,6 +7,6 @@ SPARK_VERSION: 1.0.0-SNAPSHOT SPARK_VERSION_SHORT: 1.0.0 SCALA_BINARY_VERSION: "2.10" SCALA_VERSION: "2.10.4" -MESOS_VERSION: 0.13.0 +MESOS_VERSION: 0.18.1 SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 68259f0cb8ad1..ef762aa7b8fcc 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -3,19 +3,123 @@ layout: global title: Running Spark on Mesos --- -Spark can run on clusters managed by [Apache Mesos](http://mesos.apache.org/). Follow the steps below to install Mesos and Spark: - -1. Download and build Spark using the instructions [here](index.html). **Note:** Don't forget to consider what version of HDFS you might want to use! -2. Download, build, install, and start Mesos {{site.MESOS_VERSION}} on your cluster. You can download the Mesos distribution from a [mirror](http://www.apache.org/dyn/closer.cgi/mesos/{{site.MESOS_VERSION}}/). See the Mesos [Getting Started](http://mesos.apache.org/gettingstarted) page for more information. **Note:** If you want to run Mesos without installing it into the default paths on your system (e.g., if you don't have administrative privileges to install it), you should also pass the `--prefix` option to `configure` to tell it where to install. For example, pass `--prefix=/home/user/mesos`. By default the prefix is `/usr/local`. -3. Create a Spark "distribution" using `make-distribution.sh`. -4. Rename the `dist` directory created from `make-distribution.sh` to `spark-{{site.SPARK_VERSION}}`. -5. Create a `tar` archive: `tar czf spark-{{site.SPARK_VERSION}}.tar.gz spark-{{site.SPARK_VERSION}}` -6. Upload this archive to HDFS or another place accessible from Mesos via `http://`, e.g., [Amazon Simple Storage Service](http://aws.amazon.com/s3): `hadoop fs -put spark-{{site.SPARK_VERSION}}.tar.gz /path/to/spark-{{site.SPARK_VERSION}}.tar.gz` -7. Create a file called `spark-env.sh` in Spark's `conf` directory, by copying `conf/spark-env.sh.template`, and add the following lines to it: - * `export MESOS_NATIVE_LIBRARY=`. This path is usually `/lib/libmesos.so` (where the prefix is `/usr/local` by default, see above). Also, on Mac OS X, the library is called `libmesos.dylib` instead of `libmesos.so`. - * `export SPARK_EXECUTOR_URI=`. - * `export MASTER=mesos://HOST:PORT` where HOST:PORT is the host and port (default: 5050) of your Mesos master (or `zk://...` if using Mesos with ZooKeeper). -8. To run a Spark application against the cluster, when you create your `SparkContext`, pass the string `mesos://HOST:PORT` as the master URL. In addition, you'll need to set the `spark.executor.uri` property. For example: +# Why Mesos + +Spark can run on hardware clusters managed by [Apache Mesos](http://mesos.apache.org/). + +The advantages of deploying Spark with Mesos include: +- dynamic partitioning between Spark and other + [frameworks](https://mesos.apache.org/documentation/latest/mesos-frameworks/) +- scalable partitioning between multiple instances of Spark + +# How it works + +In a standalone cluster deployment, the cluster manager in the below diagram is a Spark master +instance. When using Mesos, the Mesos master replaces the Spark master as the cluster manager. + +

      + Spark cluster components +

      + +Now when a driver creates a job and starts issuing tasks for scheduling, Mesos determines what +machines handle what tasks. Because it takes into account other frameworks when scheduling these +many short-lived tasks, multiple frameworks can coexist on the same cluster without resorting to a +static partitioning of resources. + +To get started, follow the steps below to install Mesos and deploy Spark jobs via Mesos. + + +# Installing Mesos + +Spark {{site.SPARK_VERSION}} is designed for use with Mesos {{site.MESOS_VERSION}} and does not +require any special patches of Mesos. + +If you already have a Mesos cluster running, you can skip this Mesos installation step. + +Otherwise, installing Mesos for Spark is no different than installing Mesos for use by other +frameworks. You can install Mesos using either prebuilt packages or by compiling from source. + +## Prebuilt packages + +The Apache Mesos project only publishes source package releases, no binary releases. But other +third party projects publish binary releases that may be helpful in setting Mesos up. + +One of those is Mesosphere. To install Mesos using the binary releases provided by Mesosphere: + +1. Download Mesos installation package from [downloads page](http://mesosphere.io/downloads/) +2. Follow their instructions for installation and configuration + +The Mesosphere installation documents suggest setting up ZooKeeper to handle Mesos master failover, +but Mesos can be run without ZooKeeper using a single master as well. + +## From source + +To install Mesos directly from the upstream project rather than a third party, install from source. + +1. Download the Mesos distribution from a + [mirror](http://www.apache.org/dyn/closer.cgi/mesos/{{site.MESOS_VERSION}}/) +2. Follow the Mesos [Getting Started](http://mesos.apache.org/gettingstarted) page for compiling and + installing Mesos + +**Note:** If you want to run Mesos without installing it into the default paths on your system +(e.g., if you lack administrative privileges to install it), you should also pass the +`--prefix` option to `configure` to tell it where to install. For example, pass +`--prefix=/home/user/mesos`. By default the prefix is `/usr/local`. + +## Verification + +To verify that the Mesos cluster is ready for Spark, navigate to the Mesos master webui at port +`:5050` Confirm that all expected machines are present in the slaves tab. + + +# Connecting Spark to Mesos + +To use Mesos from Spark, you need a Spark distribution available in a place accessible by Mesos, and +a Spark driver program configured to connect to Mesos. + +## Uploading Spark Distribution + +When Mesos runs a task on a Mesos slave for the first time, that slave must have a distribution of +Spark available for running the Spark Mesos executor backend. A distribution of Spark is just a +compiled binary version of Spark. + +The Spark distribution can be hosted at any Hadoop URI, including HTTP via `http://`, [Amazon Simple +Storage Service](http://aws.amazon.com/s3) via `s3://`, or HDFS via `hdfs:///`. + +To use a precompiled distribution: + +1. Download a Spark distribution from the Spark [download page](https://spark.apache.org/downloads.html) +2. Upload to hdfs/http/s3 + +To host on HDFS, use the Hadoop fs put command: `hadoop fs -put spark-{{site.SPARK_VERSION}}.tar.gz +/path/to/spark-{{site.SPARK_VERSION}}.tar.gz` + + +Or if you are using a custom-compiled version of Spark, you will need to create a distribution using +the `make-distribution.sh` script included in a Spark source tarball/checkout. + +1. Download and build Spark using the instructions [here](index.html) +2. Create a Spark distribution using `make-distribution.sh --tgz`. +3. Upload archive to http/s3/hdfs + + +## Using a Mesos Master URL + +The Master URLs for Mesos are in the form `mesos://host:5050` for a single-master Mesos +cluster, or `zk://host:2181` for a multi-master Mesos cluster using ZooKeeper. + +The driver also needs some configuration in `spark-env.sh` to interact properly with Mesos: + +1. In `spark.env.sh` set some environment variables: + * `export MESOS_NATIVE_LIBRARY=`. This path is typically + `/lib/libmesos.so` where the prefix is `/usr/local` by default. See Mesos installation + instructions above. On Mac OS X, the library is called `libmesos.dylib` instead of + `libmesos.so`. + * `export SPARK_EXECUTOR_URI=`. +2. Also set `spark.executor.uri` to + +Now when starting a Spark application against the cluster, pass a `mesos://` +or `zk://` URL as the master when creating a `SparkContext`. For example: {% highlight scala %} val conf = new SparkConf() @@ -25,31 +129,73 @@ val conf = new SparkConf() val sc = new SparkContext(conf) {% endhighlight %} +When running a shell the `spark.executor.uri` parameter is inherited from `SPARK_EXECUTOR_URI`, so +it does not need to be redundantly passed in as a system property. + +{% highlight bash %} +./bin/spark-shell --master mesos://host:5050 +{% endhighlight %} + + # Mesos Run Modes -Spark can run over Mesos in two modes: "fine-grained" and "coarse-grained". In fine-grained mode, which is the default, -each Spark task runs as a separate Mesos task. This allows multiple instances of Spark (and other frameworks) to share -machines at a very fine granularity, where each application gets more or fewer machines as it ramps up, but it comes with an -additional overhead in launching each task, which may be inappropriate for low-latency applications (e.g. interactive queries or serving web requests). The coarse-grained mode will instead -launch only *one* long-running Spark task on each Mesos machine, and dynamically schedule its own "mini-tasks" within -it. The benefit is much lower startup overhead, but at the cost of reserving the Mesos resources for the complete duration -of the application. +Spark can run over Mesos in two modes: "fine-grained" (default) and "coarse-grained". + +In "fine-grained" mode (default), each Spark task runs as a separate Mesos task. This allows +multiple instances of Spark (and other frameworks) to share machines at a very fine granularity, +where each application gets more or fewer machines as it ramps up and down, but it comes with an +additional overhead in launching each task. This mode may be inappropriate for low-latency +requirements like interactive queries or serving web requests. + +The "coarse-grained" mode will instead launch only *one* long-running Spark task on each Mesos +machine, and dynamically schedule its own "mini-tasks" within it. The benefit is much lower startup +overhead, but at the cost of reserving the Mesos resources for the complete duration of the +application. -To run in coarse-grained mode, set the `spark.mesos.coarse` property in your [SparkConf](configuration.html#spark-properties): +To run in coarse-grained mode, set the `spark.mesos.coarse` property in your +[SparkConf](configuration.html#spark-properties): {% highlight scala %} conf.set("spark.mesos.coarse", "true") {% endhighlight %} -In addition, for coarse-grained mode, you can control the maximum number of resources Spark will acquire. By default, -it will acquire *all* cores in the cluster (that get offered by Mesos), which only makes sense if you run just one -application at a time. You can cap the maximum number of cores using `conf.set("spark.cores.max", "10")` (for example). +In addition, for coarse-grained mode, you can control the maximum number of resources Spark will +acquire. By default, it will acquire *all* cores in the cluster (that get offered by Mesos), which +only makes sense if you run just one application at a time. You can cap the maximum number of cores +using `conf.set("spark.cores.max", "10")` (for example). # Running Alongside Hadoop -You can run Spark and Mesos alongside your existing Hadoop cluster by just launching them as a separate service on the machines. To access Hadoop data from Spark, just use a hdfs:// URL (typically `hdfs://:9000/path`, but you can find the right URL on your Hadoop Namenode's web UI). +You can run Spark and Mesos alongside your existing Hadoop cluster by just launching them as a +separate service on the machines. To access Hadoop data from Spark, a full hdfs:// URL is required +(typically `hdfs://:9000/path`, but you can find the right URL on your Hadoop Namenode web +UI). + +In addition, it is possible to also run Hadoop MapReduce on Mesos for better resource isolation and +sharing between the two. In this case, Mesos will act as a unified scheduler that assigns cores to +either Hadoop or Spark, as opposed to having them share resources via the Linux scheduler on each +node. Please refer to [Hadoop on Mesos](https://github.com/mesos/hadoop). + +In either case, HDFS runs separately from Hadoop MapReduce, without being scheduled through Mesos. + + +# Troubleshooting and Debugging + +A few places to look during debugging: + +- Mesos master on port `:5050` + - Slaves should appear in the slaves tab + - Spark applications should appear in the frameworks tab + - Tasks should appear in the details of a framework + - Check the stdout and stderr of the sandbox of failed tasks +- Mesos logs + - Master and slave logs are both in `/var/log/mesos` by default -In addition, it is possible to also run Hadoop MapReduce on Mesos, to get better resource isolation and sharing between the two. In this case, Mesos will act as a unified scheduler that assigns cores to either Hadoop or Spark, as opposed to having them share resources via the Linux scheduler on each node. Please refer to [Hadoop on Mesos](https://github.com/mesos/hadoop). +And common pitfalls: -In either case, HDFS runs separately from Hadoop MapReduce, without going through Mesos. +- Spark assembly not reachable/accessible + - Slaves need to be able to download the distribution +- Firewall blocking communications + - Check for messages about failed connections + - Temporarily disable firewalls for debugging and then poke appropriate holes From d58cb33ffa9e98a64cecea7b40ce7bfbed145079 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 14 May 2014 09:51:01 -0700 Subject: [PATCH 350/641] SPARK-1828: Created forked version of hive-exec that doesn't bundle other dependencies See https://issues.apache.org/jira/browse/SPARK-1828 for more information. This is being submitted to Jenkin's for testing. The dependency won't fully propagate in Maven central for a few more hours. Author: Patrick Wendell Closes #767 from pwendell/hive-shaded and squashes the following commits: ea10ac5 [Patrick Wendell] SPARK-1828: Created forked version of hive-exec that doesn't bundle other dependencies --- project/SparkBuild.scala | 6 +++--- sql/hive/pom.xml | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 57b3e22f81713..8d56b40431388 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -489,9 +489,9 @@ object SparkBuild extends Build { name := "spark-hive", javaOptions += "-XX:MaxPermSize=1g", libraryDependencies ++= Seq( - "org.apache.hive" % "hive-metastore" % hiveVersion, - "org.apache.hive" % "hive-exec" % hiveVersion, - "org.apache.hive" % "hive-serde" % hiveVersion + "org.spark-project.hive" % "hive-metastore" % hiveVersion, + "org.spark-project.hive" % "hive-exec" % hiveVersion, + "org.spark-project.hive" % "hive-serde" % hiveVersion ), // Multiple queries rely on the TestHive singleton. See comments there for more details. parallelExecution in Test := false, diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 8b32451d76045..9254b70e64a08 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -44,12 +44,12 @@ ${project.version}
      - org.apache.hive + org.spark-project.hive hive-metastore ${hive.version} - org.apache.hive + org.spark-project.hive hive-exec ${hive.version} @@ -64,7 +64,7 @@ jackson-mapper-asl - org.apache.hive + org.spark-project.hive hive-serde ${hive.version} From 17f3075bc4aa8cbed165f7b367f70e84b1bc8db9 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 14 May 2014 10:07:25 -0700 Subject: [PATCH 351/641] [SPARK-1620] Handle uncaught exceptions in function run by Akka scheduler If the intended behavior was that uncaught exceptions thrown in functions being run by the Akka scheduler would end up being handled by the default uncaught exception handler set in Executor, and if that behavior is, in fact, correct, then this is a way to accomplish that. I'm not certain, though, that we shouldn't be doing something different to handle uncaught exceptions from some of these scheduled functions. In any event, this PR covers all of the cases I comment on in [SPARK-1620](https://issues.apache.org/jira/browse/SPARK-1620). Author: Mark Hamstra Closes #622 from markhamstra/SPARK-1620 and squashes the following commits: 071d193 [Mark Hamstra] refactored post-SPARK-1772 1a6a35e [Mark Hamstra] another style fix d30eb94 [Mark Hamstra] scalastyle 3573ecd [Mark Hamstra] Use wrapped try/catch in Utils.tryOrExit 8fc0439 [Mark Hamstra] Make functions run by the Akka scheduler use Executor's UncaughtExceptionHandler --- .../apache/spark/deploy/client/AppClient.scala | 18 ++++++++++-------- .../apache/spark/deploy/worker/Worker.scala | 18 ++++++++++-------- .../spark/scheduler/TaskSchedulerImpl.scala | 3 ++- .../apache/spark/storage/BlockManager.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 13 +++++++++++++ 5 files changed, 36 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 896913d796604..d38e9e79204c2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -30,7 +30,7 @@ import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.{Utils, AkkaUtils} /** * Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL, @@ -88,13 +88,15 @@ private[spark] class AppClient( var retries = 0 registrationRetryTimer = Some { context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { - retries += 1 - if (registered) { - registrationRetryTimer.foreach(_.cancel()) - } else if (retries >= REGISTRATION_RETRIES) { - markDead("All masters are unresponsive! Giving up.") - } else { - tryRegisterAllMasters() + Utils.tryOrExit { + retries += 1 + if (registered) { + registrationRetryTimer.foreach(_.cancel()) + } else if (retries >= REGISTRATION_RETRIES) { + markDead("All masters are unresponsive! Giving up.") + } else { + tryRegisterAllMasters() + } } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 85d25dc7dbfa4..134624c35a57e 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -166,14 +166,16 @@ private[spark] class Worker( var retries = 0 registrationRetryTimer = Some { context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { - retries += 1 - if (registered) { - registrationRetryTimer.foreach(_.cancel()) - } else if (retries >= REGISTRATION_RETRIES) { - logError("All masters are unresponsive! Giving up.") - System.exit(1) - } else { - tryRegisterAllMasters() + Utils.tryOrExit { + retries += 1 + if (registered) { + registrationRetryTimer.foreach(_.cancel()) + } else if (retries >= REGISTRATION_RETRIES) { + logError("All masters are unresponsive! Giving up.") + System.exit(1) + } else { + tryRegisterAllMasters() + } } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index ffd1d9432682b..649eed213e09a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -31,6 +31,7 @@ import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.SchedulingMode.SchedulingMode +import org.apache.spark.util.Utils /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. @@ -139,7 +140,7 @@ private[spark] class TaskSchedulerImpl( import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, SPECULATION_INTERVAL milliseconds) { - checkSpeculatableTasks() + Utils.tryOrExit { checkSpeculatableTasks() } } } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 6d7d4f922e1fa..6534095811907 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -155,7 +155,7 @@ private[spark] class BlockManager( BlockManagerWorker.startBlockManagerWorker(this) if (!BlockManager.getDisableHeartBeatsForTesting(conf)) { heartBeatTask = actorSystem.scheduler.schedule(0.seconds, heartBeatFrequency.milliseconds) { - heartBeat() + Utils.tryOrExit { heartBeat() } } } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 99ef6dd1fa6e3..d041bfa66a1cd 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -40,6 +40,7 @@ import tachyon.client.{TachyonFile,TachyonFS} import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.executor.ExecutorUncaughtExceptionHandler import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} /** @@ -780,6 +781,18 @@ private[spark] object Utils extends Logging { output.toString } + /** + * Execute a block of code that evaluates to Unit, forwarding any uncaught exceptions to the + * default UncaughtExceptionHandler + */ + def tryOrExit(block: => Unit) { + try { + block + } catch { + case t: Throwable => ExecutorUncaughtExceptionHandler.uncaughtException(t) + } + } + /** * A regular expression to match classes of the "core" Spark API that we want to skip when * finding the call site of a method. From fde82c1549c78f1eebbb21ec34e60befbbff65f5 Mon Sep 17 00:00:00 2001 From: witgo Date: Wed, 14 May 2014 11:19:26 -0700 Subject: [PATCH 352/641] Fix: sbt test throw an java.lang.OutOfMemoryError: PermGen space Author: witgo Closes #773 from witgo/sbt_javaOptions and squashes the following commits: 26c7d38 [witgo] Improve sbt configuration --- .rat-excludes | 5 +++++ project/SparkBuild.scala | 1 + 2 files changed, 6 insertions(+) diff --git a/.rat-excludes b/.rat-excludes index 50766954ef070..689467847d80d 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -43,3 +43,8 @@ test.out/* .*iml service.properties db.lck +build/* +dist/* +.*out +.*ipr +.*iws diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 8d56b40431388..6adec556322ba 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -183,6 +183,7 @@ object SparkBuild extends Build { javaOptions in Test += "-Dspark.testing=1", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark").map { case (k,v) => s"-D$k=$v" }.toSeq, + javaOptions in Test ++= "-Xmx3g -XX:PermSize=128M -XX:MaxNewSize=256m -XX:MaxPermSize=1g".split(" ").toSeq, javaOptions += "-Xmx3g", // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), From a3315d7f4c7584dae2ee0aa33c6ec9e97b229b48 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Wed, 14 May 2014 12:01:14 -0700 Subject: [PATCH 353/641] SPARK-1829 Sub-second durations shouldn't round to "0 s" As "99 ms" up to 99 ms As "0.1 s" from 0.1 s up to 0.9 s https://issues.apache.org/jira/browse/SPARK-1829 Compare the first image to the second here: http://imgur.com/RaLEsSZ,7VTlgfo#0 Author: Andrew Ash Closes #768 from ash211/spark-1829 and squashes the following commits: 1c15b8e [Andrew Ash] SPARK-1829 Format sub-second durations more appropriately --- core/src/main/scala/org/apache/spark/ui/UIUtils.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index a3d6a1821245b..a43314f48112f 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -36,7 +36,13 @@ private[spark] object UIUtils extends Logging { def formatDate(timestamp: Long): String = dateFormat.get.format(new Date(timestamp)) def formatDuration(milliseconds: Long): String = { + if (milliseconds < 100) { + return "%d ms".format(milliseconds) + } val seconds = milliseconds.toDouble / 1000 + if (seconds < 1) { + return "%.1f s".format(seconds) + } if (seconds < 60) { return "%.0f s".format(seconds) } From 65533c7ec03e7eedf5cd9756822863ab6f034ec9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 14 May 2014 12:53:30 -0700 Subject: [PATCH 354/641] SPARK-1833 - Have an empty SparkContext constructor. This is nicer than relying on new SparkContext(new SparkConf()) Author: Patrick Wendell Closes #774 from pwendell/spark-context and squashes the following commits: ef9f12f [Patrick Wendell] SPARK-1833 - Have an empty SparkContext constructor. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 032b3d744c619..634c10c33fc86 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -66,6 +66,12 @@ class SparkContext(config: SparkConf) extends Logging { // contains a map from hostname to a list of input format splits on the host. private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map() + /** + * Create a SparkContext that loads settings from system properties (for instance, when + * launching with ./bin/spark-submit). + */ + def this() = this(new SparkConf()) + /** * :: DeveloperApi :: * Alternative constructor for setting preferred locations where Spark will create executors. From 94c6c06ea13032b80610b3f54401d2ef2aa4874a Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 14 May 2014 14:57:17 -0700 Subject: [PATCH 355/641] [FIX] do not load defaults when testing SparkConf in pyspark The default constructor loads default properties, which can fail the test. Author: Xiangrui Meng Closes #775 from mengxr/pyspark-conf-fix and squashes the following commits: 83ef6c4 [Xiangrui Meng] do not load defaults when testing SparkConf in pyspark --- python/pyspark/conf.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index 49b68d57ab02c..8eff4a242a529 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -33,7 +33,7 @@ >>> sc.sparkHome == None True ->>> conf = SparkConf() +>>> conf = SparkConf(loadDefaults=False) >>> conf.setSparkHome("/path") >>> conf.get("spark.home") From 601e37198b97ba52e72ac13213c391c932e97b67 Mon Sep 17 00:00:00 2001 From: Jacek Laskowski Date: Wed, 14 May 2014 15:45:52 -0700 Subject: [PATCH 356/641] String interpolation + some other small changes After having been invited to make the change in https://github.com/apache/spark/commit/6bee01dd04ef73c6b829110ebcdd622d521ea8ff#commitcomment-6284165 by @witgo. Author: Jacek Laskowski Closes #748 from jaceklaskowski/sparkenv-string-interpolation and squashes the following commits: be6ebac [Jacek Laskowski] String interpolation + some other small changes --- .../main/scala/org/apache/spark/SparkEnv.scala | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 19d507c0cf860..720151a6b0f84 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -278,10 +278,11 @@ object SparkEnv extends Logging { addedJars: Seq[String], addedFiles: Seq[String]): Map[String, Seq[(String, String)]] = { + import Properties._ val jvmInformation = Seq( - ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), - ("Java Home", Properties.javaHome), - ("Scala Version", Properties.versionString) + ("Java Version", s"$javaVersion ($javaVendor)"), + ("Java Home", javaHome), + ("Scala Version", versionString) ).sorted // Spark properties @@ -296,18 +297,15 @@ object SparkEnv extends Logging { // System properties that are not java classpaths val systemProperties = System.getProperties.iterator.toSeq - val otherProperties = systemProperties.filter { case (k, v) => + val otherProperties = systemProperties.filter { case (k, _) => k != "java.class.path" && !k.startsWith("spark.") }.sorted // Class paths including all added jars and files - val classPathProperty = systemProperties.find { case (k, v) => - k == "java.class.path" - }.getOrElse(("", "")) - val classPathEntries = classPathProperty._2 + val classPathEntries = javaClassPath .split(File.pathSeparator) - .filterNot(e => e.isEmpty) - .map(e => (e, "System Classpath")) + .filterNot(_.isEmpty) + .map((_, "System Classpath")) val addedJarsAndFiles = (addedJars ++ addedFiles).map((_, "Added By User")) val classPaths = (addedJarsAndFiles ++ classPathEntries).sorted From e3d72a74ad007c2bf279d6a74cdaca948bdf0ddd Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 14 May 2014 17:18:30 -0700 Subject: [PATCH 357/641] [SPARK-1696][MLLIB] use alpha in dense dspr It doesn't affect existing code because only `alpha = 1.0` is used in the code. Author: Xiangrui Meng Closes #778 from mengxr/mllib-dspr-fix and squashes the following commits: a37402e [Xiangrui Meng] use alpha in dense dspr --- .../org/apache/spark/mllib/linalg/distributed/RowMatrix.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index b10857fe7c8a1..07dfadf2f7869 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -440,7 +440,7 @@ object RowMatrix { val n = v.size v match { case dv: DenseVector => - blas.dspr("U", n, 1.0, dv.values, 1, U) + blas.dspr("U", n, alpha, dv.values, 1, U) case sv: SparseVector => val indices = sv.indices val values = sv.values From 9ad096d55a3d8410f04056ebc87dbd8cba391870 Mon Sep 17 00:00:00 2001 From: andrewor14 Date: Wed, 14 May 2014 17:54:53 -0700 Subject: [PATCH 358/641] [Typo] propertes -> properties Author: andrewor14 Closes #780 from andrewor14/submit-typo and squashes the following commits: e70e057 [andrewor14] propertes -> properties --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index c6d3cbd2e728b..e86182e4c56ce 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -68,7 +68,7 @@ object SparkSubmit { /** * @return a tuple containing the arguments for the child, a list of classpath - * entries for the child, a list of system propertes, a list of env vars + * entries for the child, a list of system properties, a list of env vars * and the main class for the child */ private[spark] def createLaunchEnv(args: SparkSubmitArguments): (ArrayBuffer[String], From 44165fc91a31e6293a79031c89571e139d2c5356 Mon Sep 17 00:00:00 2001 From: wangfei Date: Wed, 14 May 2014 17:59:11 -0700 Subject: [PATCH 359/641] [SPARK-1826] fix the head notation of package object dsl Author: wangfei Closes #765 from scwf/dslfix and squashes the following commits: d2d1a9d [wangfei] Update package.scala 66ff53b [wangfei] fix the head notation of package object dsl --- .../spark/sql/catalyst/dsl/package.scala | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 78d3a1d8096af..3cf163f9a9a75 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -31,26 +31,29 @@ import org.apache.spark.sql.catalyst.types._ * A collection of implicit conversions that create a DSL for constructing catalyst data structures. * * {{{ - * scala> import catalyst.dsl._ + * scala> import org.apache.spark.sql.catalyst.dsl.expressions._ * * // Standard operators are added to expressions. + * scala> import org.apache.spark.sql.catalyst.expressions.Literal * scala> Literal(1) + Literal(1) - * res1: catalyst.expressions.Add = (1 + 1) + * res0: org.apache.spark.sql.catalyst.expressions.Add = (1 + 1) * * // There is a conversion from 'symbols to unresolved attributes. * scala> 'a.attr - * res2: catalyst.analysis.UnresolvedAttribute = 'a + * res1: org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute = 'a * * // These unresolved attributes can be used to create more complicated expressions. * scala> 'a === 'b - * res3: catalyst.expressions.Equals = ('a = 'b) + * res2: org.apache.spark.sql.catalyst.expressions.Equals = ('a = 'b) * * // SQL verbs can be used to construct logical query plans. - * scala> TestRelation('key.int, 'value.string).where('key === 1).select('value).analyze - * res4: catalyst.plans.logical.LogicalPlan = - * Project {value#1} - * Filter (key#0 = 1) - * TestRelation {key#0,value#1} + * scala> import org.apache.spark.sql.catalyst.plans.logical._ + * scala> import org.apache.spark.sql.catalyst.dsl.plans._ + * scala> LocalRelation('key.int, 'value.string).where('key === 1).select('value).analyze + * res3: org.apache.spark.sql.catalyst.plans.logical.LogicalPlan = + * Project [value#3] + * Filter (key#2 = 1) + * LocalRelation [key#2,value#3], [] * }}} */ package object dsl { From 2f639957f0bf70dddf1e698aa9e26007fb58bc67 Mon Sep 17 00:00:00 2001 From: Chen Chao Date: Wed, 14 May 2014 18:20:20 -0700 Subject: [PATCH 360/641] default task number misleading in several places private[streaming] def defaultPartitioner(numPartitions: Int = self.ssc.sc.defaultParallelism){ new HashPartitioner(numPartitions) } it represents that the default task number in Spark Streaming relies on the variable defaultParallelism in SparkContext, which is decided by the config property spark.default.parallelism the property "spark.default.parallelism" refers to https://github.com/apache/spark/pull/389 Author: Chen Chao Closes #766 from CrazyJvm/patch-7 and squashes the following commits: 0b7efba [Chen Chao] Update streaming-programming-guide.md cc5b66c [Chen Chao] default task number misleading in several places --- docs/streaming-programming-guide.md | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 939599aa6855b..0c125eb693a8e 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -522,9 +522,9 @@ common ones are as follows. reduceByKey(func, [numTasks]) When called on a DStream of (K, V) pairs, return a new DStream of (K, V) pairs where the values for each key are aggregated using the given reduce function. Note: By default, - this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluster) to - do the grouping. You can pass an optional numTasks argument to set a different - number of tasks. + this uses Spark's default number of parallel tasks (2 for local mode, and in cluster mode the number + is determined by the config property spark.default.parallelism) to do the grouping. + You can pass an optional numTasks argument to set a different number of tasks. join(otherStream, [numTasks]) @@ -743,8 +743,9 @@ said two parameters - windowLength and slideInterval. When called on a DStream of (K, V) pairs, returns a new DStream of (K, V) pairs where the values for each key are aggregated using the given reduce function func over batches in a sliding window. Note: By default, this uses Spark's default number of - parallel tasks (2 for local machine, 8 for a cluster) to do the grouping. You can pass an optional - numTasks argument to set a different number of tasks. + parallel tasks (2 for local mode, and in cluster mode the number is determined by the config + property spark.default.parallelism) to do the grouping. You can pass an optional + numTasks argument to set a different number of tasks. @@ -956,9 +957,10 @@ before further processing. ### Level of Parallelism in Data Processing Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` -and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of -parallelism as an argument (see the -[`PairDStreamFunctions`](api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions) +and `reduceByKeyAndWindow`, the default number of parallel tasks is decided by the [config property] +(configuration.html#spark-properties) `spark.default.parallelism`. You can pass the level of +parallelism as an argument (see [`PairDStreamFunctions`] +(api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions) documentation), or set the [config property](configuration.html#spark-properties) `spark.default.parallelism` to change the default. From ad4e60ee7e2c49c24a9972312915f7f7253c7679 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 14 May 2014 21:13:41 -0700 Subject: [PATCH 361/641] [SPARK-1840] SparkListenerBus prints out scary error message when terminated normally Running SparkPi example gave this error. ``` Pi is roughly 3.14374 14/05/14 18:16:19 ERROR Utils: Uncaught exception in thread SparkListenerBus scala.runtime.NonLocalReturnControl$mcV$sp ``` This is due to the catch-all in the SparkListenerBus, which logged control throwable used by scala system Author: Tathagata Das Closes #783 from tdas/controlexception-fix and squashes the following commits: a466c8d [Tathagata Das] Ignored control exceptions when logging all exceptions. --- core/src/main/scala/org/apache/spark/util/Utils.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index d041bfa66a1cd..388f7222428db 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1149,6 +1149,8 @@ private[spark] object Utils extends Logging { try { f } catch { + case ct: ControlThrowable => + throw ct case t: Throwable => logError(s"Uncaught exception in thread ${Thread.currentThread().getName}", t) throw t From f10de042b8e86adf51b70bae2d8589a5cbf02935 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 14 May 2014 21:45:20 -0700 Subject: [PATCH 362/641] Add language tabs and Python version to interactive part of quick-start This is an addition of some stuff that was missed in https://issues.apache.org/jira/browse/SPARK-1567. I've also updated the doc to show submitting the Python application with spark-submit. Author: Matei Zaharia Closes #782 from mateiz/spark-1567-extra and squashes the following commits: 6f8f2aa [Matei Zaharia] tweaks 9ed9874 [Matei Zaharia] tweaks ae67c3e [Matei Zaharia] tweak b303ba3 [Matei Zaharia] tweak 1433a4d [Matei Zaharia] Add language tabs and Python version to interactive part of quick-start guide --- docs/python-programming-guide.md | 2 +- docs/quick-start.md | 151 +++++++++++++++++++++++++++---- 2 files changed, 133 insertions(+), 20 deletions(-) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 2ce2c346d7b04..17675acba6bb8 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -121,7 +121,7 @@ IPython also works on a cluster or on multiple cores if you set the `MASTER` env # Standalone Programs PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `bin/spark-submit`. -The Quick Start guide includes a [complete example](quick-start.html#a-standalone-app-in-python) of a standalone Python application. +The Quick Start guide includes a [complete example](quick-start.html#standalone-applications) of a standalone Python application. Code dependencies can be deployed by passing .zip or .egg files in the `--py-files` option of `spark-submit`: diff --git a/docs/quick-start.md b/docs/quick-start.md index a4d01487bb494..33a0df1036424 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -6,7 +6,9 @@ title: Quick Start * This will become a table of contents (this text will be scraped). {:toc} -This tutorial provides a quick introduction to using Spark. We will first introduce the API through Spark's interactive Scala shell (don't worry if you don't know Scala -- you will not need much for this), then show how to write standalone applications in Scala, Java, and Python. +This tutorial provides a quick introduction to using Spark. We will first introduce the API through Spark's +interactive shell (in Python or Scala), +then show how to write standalone applications in Java, Scala, and Python. See the [programming guide](scala-programming-guide.html) for a more complete reference. To follow along with this guide, first download a packaged release of Spark from the @@ -17,8 +19,12 @@ you can download a package for any version of Hadoop. ## Basics -Spark's interactive shell provides a simple way to learn the API, as well as a powerful tool to analyze datasets interactively. -Start the shell by running the following in the Spark directory. +Spark's shell provides a simple way to learn the API, as well as a powerful tool to analyze data interactively. +It is available in either Scala (which runs on the Java VM and is thus a good way to use existing Java libraries) +or Python. Start it by running the following in the Spark directory: + +
      +
      ./bin/spark-shell @@ -33,7 +39,7 @@ RDDs have _[actions](scala-programming-guide.html#actions)_, which return values {% highlight scala %} scala> textFile.count() // Number of items in this RDD -res0: Long = 74 +res0: Long = 126 scala> textFile.first() // First item in this RDD res1: String = # Apache Spark @@ -53,12 +59,53 @@ scala> textFile.filter(line => line.contains("Spark")).count() // How many lines res3: Long = 15 {% endhighlight %} +
      +
      + + ./bin/pyspark + +Spark's primary abstraction is a distributed collection of items called a Resilient Distributed Dataset (RDD). RDDs can be created from Hadoop InputFormats (such as HDFS files) or by transforming other RDDs. Let's make a new RDD from the text of the README file in the Spark source directory: + +{% highlight python %} +>>> textFile = sc.textFile("README.md") +{% endhighlight %} + +RDDs have _[actions](scala-programming-guide.html#actions)_, which return values, and _[transformations](scala-programming-guide.html#transformations)_, which return pointers to new RDDs. Let's start with a few actions: + +{% highlight python %} +>>> textFile.count() # Number of items in this RDD +126 + +>>> textFile.first() # First item in this RDD +u'# Apache Spark' +{% endhighlight %} + +Now let's use a transformation. We will use the [`filter`](scala-programming-guide.html#transformations) transformation to return a new RDD with a subset of the items in the file. + +{% highlight python %} +>>> linesWithSpark = textFile.filter(lambda line: "Spark" in line) +{% endhighlight %} + +We can chain together transformations and actions: + +{% highlight python %} +>>> textFile.filter(lambda line: "Spark" in line).count() # How many lines contain "Spark"? +15 +{% endhighlight %} + +
      +
      + + ## More on RDD Operations RDD actions and transformations can be used for more complex computations. Let's say we want to find the line with the most words: +
      +
      + {% highlight scala %} scala> textFile.map(line => line.split(" ").size).reduce((a, b) => if (a > b) a else b) -res4: Long = 16 +res4: Long = 15 {% endhighlight %} This first maps a line to an integer value, creating a new RDD. `reduce` is called on that RDD to find the largest line count. The arguments to `map` and `reduce` are Scala function literals (closures), and can use any language feature or Scala/Java library. For example, we can easily call functions declared elsewhere. We'll use `Math.max()` function to make this code easier to understand: @@ -68,26 +115,69 @@ scala> import java.lang.Math import java.lang.Math scala> textFile.map(line => line.split(" ").size).reduce((a, b) => Math.max(a, b)) -res5: Int = 16 +res5: Int = 15 {% endhighlight %} One common data flow pattern is MapReduce, as popularized by Hadoop. Spark can implement MapReduce flows easily: {% highlight scala %} scala> val wordCounts = textFile.flatMap(line => line.split(" ")).map(word => (word, 1)).reduceByKey((a, b) => a + b) -wordCounts: spark.RDD[(java.lang.String, Int)] = spark.ShuffledAggregatedRDD@71f027b8 +wordCounts: spark.RDD[(String, Int)] = spark.ShuffledAggregatedRDD@71f027b8 {% endhighlight %} Here, we combined the [`flatMap`](scala-programming-guide.html#transformations), [`map`](scala-programming-guide.html#transformations) and [`reduceByKey`](scala-programming-guide.html#transformations) transformations to compute the per-word counts in the file as an RDD of (String, Int) pairs. To collect the word counts in our shell, we can use the [`collect`](scala-programming-guide.html#actions) action: {% highlight scala %} scala> wordCounts.collect() -res6: Array[(java.lang.String, Int)] = Array((need,2), ("",43), (Extra,3), (using,1), (passed,1), (etc.,1), (its,1), (`/usr/local/lib/libmesos.so`,1), (`SCALA_HOME`,1), (option,1), (these,1), (#,1), (`PATH`,,2), (200,1), (To,3),... +res6: Array[(String, Int)] = Array((means,1), (under,2), (this,3), (Because,1), (Python,2), (agree,1), (cluster.,1), ...) {% endhighlight %} +
      +
      + +{% highlight python %} +>>> textFile.map(lambda line: len(line.split())).reduce(lambda a, b: a if (a > b) else b) +15 +{% endhighlight %} + +This first maps a line to an integer value, creating a new RDD. `reduce` is called on that RDD to find the largest line count. The arguments to `map` and `reduce` are Python [anonymous functions (lambdas)](https://docs.python.org/2/reference/expressions.html#lambda), +but we can also pass any top-level Python function we want. +For example, we'll define a `max` function to make this code easier to understand: + +{% highlight python %} +>>> def max(a, b): +... if a > b: +... return a +... else: +... return b +... + +>>> textFile.map(lambda line: len(line.split())).reduce(max) +15 +{% endhighlight %} + +One common data flow pattern is MapReduce, as popularized by Hadoop. Spark can implement MapReduce flows easily: + +{% highlight python %} +>>> wordCounts = textFile.flatMap(lambda line: line.split()).map(lambda word: (word, 1)).reduceByKey(lambda a, b: a+b) +{% endhighlight %} + +Here, we combined the [`flatMap`](scala-programming-guide.html#transformations), [`map`](scala-programming-guide.html#transformations) and [`reduceByKey`](scala-programming-guide.html#transformations) transformations to compute the per-word counts in the file as an RDD of (string, int) pairs. To collect the word counts in our shell, we can use the [`collect`](scala-programming-guide.html#actions) action: + +{% highlight python %} +>>> wordCounts.collect() +[(u'and', 9), (u'A', 1), (u'webpage', 1), (u'README', 1), (u'Note', 1), (u'"local"', 1), (u'variable', 1), ...] +{% endhighlight %} + +
      +
      + ## Caching Spark also supports pulling data sets into a cluster-wide in-memory cache. This is very useful when data is accessed repeatedly, such as when querying a small "hot" dataset or when running an iterative algorithm like PageRank. As a simple example, let's mark our `linesWithSpark` dataset to be cached: +
      +
      + {% highlight scala %} scala> linesWithSpark.cache() res7: spark.RDD[String] = spark.FilteredRDD@17e51082 @@ -99,12 +189,33 @@ scala> linesWithSpark.count() res9: Long = 15 {% endhighlight %} -It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is +It may seem silly to use Spark to explore and cache a 100-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark). -# A Standalone Application +
      +
      + +{% highlight python %} +>>> linesWithSpark.cache() + +>>> linesWithSpark.count() +15 + +>>> linesWithSpark.count() +15 +{% endhighlight %} + +It may seem silly to use Spark to explore and cache a 100-line text file. The interesting part is +that these same functions can be used on very large data sets, even when they are striped across +tens or hundreds of nodes. You can also do this interactively by connecting `bin/pyspark` to +a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark). + +
      +
      + +# Standalone Applications Now say we wanted to write a standalone application using the Spark API. We will walk through a simple application in both Scala (with SBT), Java (with Maven), and Python. @@ -115,7 +226,7 @@ We'll create a very simple Spark application in Scala. So simple, in fact, that named `SimpleApp.scala`: {% highlight scala %} -/*** SimpleApp.scala ***/ +/* SimpleApp.scala */ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ import org.apache.spark.SparkConf @@ -194,7 +305,7 @@ This example will use Maven to compile an application jar, but any similar build We'll create a very simple Spark application, `SimpleApp.java`: {% highlight java %} -/*** SimpleApp.java ***/ +/* SimpleApp.java */ import org.apache.spark.api.java.*; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.Function; @@ -309,16 +420,18 @@ Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark i As with the Scala and Java examples, we use a SparkContext to create RDDs. We can pass Python functions to Spark, which are automatically serialized along with any variables that they reference. -For applications that use custom classes or third-party libraries, we can add those code -dependencies to SparkContext to ensure that they will be available on remote machines; this is -described in more detail in the [Python programming guide](python-programming-guide.html). +For applications that use custom classes or third-party libraries, we can also add code +dependencies to `spark-submit` through its `--py-files` argument by packaging them into a +.zip file (see `spark-submit --help` for details). `SimpleApp` is simple enough that we do not need to specify any code dependencies. -We can run this application using the `bin/pyspark` script: +We can run this application using the `bin/spark-submit` script: {% highlight python %} -$ cd $SPARK_HOME -$ ./bin/pyspark SimpleApp.py +# Use spark-submit to run your application +$ YOUR_SPARK_HOME/bin/spark-submit \ + --master local[4] \ + SimpleApp.py ... Lines with a: 46, Lines with b: 23 {% endhighlight python %} @@ -326,7 +439,7 @@ Lines with a: 46, Lines with b: 23
      -# Where to go from here +# Where to Go from Here Congratulations on running your first Spark application! * For an in-depth overview of the API see "Programming Guides" menu section. From 21570b463388194877003318317aafd842800cac Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 14 May 2014 22:24:04 -0700 Subject: [PATCH 363/641] Documentation: Encourage use of reduceByKey instead of groupByKey. Author: Patrick Wendell Closes #784 from pwendell/group-by-key and squashes the following commits: 9b4505f [Patrick Wendell] Small fix 6347924 [Patrick Wendell] Documentation: Encourage use of reduceByKey instead of groupByKey. --- .../org/apache/spark/api/java/JavaPairRDD.scala | 12 ++++++++++++ .../org/apache/spark/rdd/PairRDDFunctions.scala | 12 ++++++++++++ docs/scala-programming-guide.md | 4 ++++ python/pyspark/rdd.py | 4 ++++ 4 files changed, 32 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 554c065358648..4c8f9ed6fbc02 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -263,6 +263,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[JavaPairRDD.reduceByKey]] or [[JavaPairRDD.combineByKey]] + * will provide much better performance. */ def groupByKey(partitioner: Partitioner): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(partitioner))) @@ -270,6 +274,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[JavaPairRDD.reduceByKey]] or [[JavaPairRDD.combineByKey]] + * will provide much better performance. */ def groupByKey(numPartitions: Int): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(numPartitions))) @@ -380,6 +388,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[JavaPairRDD.reduceByKey]] or [[JavaPairRDD.combineByKey]] + * will provide much better performance. */ def groupByKey(): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey())) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index bc6d204434ad8..223fef79261d0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -264,6 +264,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]] + * will provide much better performance. */ def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = { // groupByKey shouldn't use map side combine because map side combine does not @@ -280,6 +284,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]] + * will provide much better performance. */ def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = { groupByKey(new HashPartitioner(numPartitions)) @@ -365,6 +373,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]] + * will provide much better performance, */ def groupByKey(): RDD[(K, Iterable[V])] = { groupByKey(defaultPartitioner(self)) diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 3ed86e460c01c..edaa7d0639a18 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -196,6 +196,10 @@ The following tables list the transformations and actions currently supported (s groupByKey([numTasks]) When called on a dataset of (K, V) pairs, returns a dataset of (K, Seq[V]) pairs.
      +Note: If you are grouping in order to perform an aggregation (such as a sum or + average) over each key, using `reduceByKey` or `combineByKey` will yield much better + performance. +
      Note: By default, if the RDD already has a partitioner, the task number is decided by the partition number of the partitioner, or else relies on the value of spark.default.parallelism if the property is set , otherwise depends on the partition number of the RDD. You can pass an optional numTasks argument to set a different number of tasks. diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 4f74824ba4cf2..07578b8d937fc 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1152,6 +1152,10 @@ def groupByKey(self, numPartitions=None): Group the values for each key in the RDD into a single sequence. Hash-partitions the resulting RDD with into numPartitions partitions. + Note: If you are grouping in order to perform an aggregation (such as a + sum or average) over each key, using reduceByKey will provide much better + performance. + >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) >>> map((lambda (x,y): (x, list(y))), sorted(x.groupByKey().collect())) [('a', [1, 1]), ('b', [1])] From 46324279dae2fa803267d788f7c56b0ed643b4c8 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 14 May 2014 22:24:41 -0700 Subject: [PATCH 364/641] Package docs This is a few changes based on the original patch by @scrapcodes. Author: Prashant Sharma Author: Patrick Wendell Closes #785 from pwendell/package-docs and squashes the following commits: c32b731 [Patrick Wendell] Changes based on Prashant's patch c0463d3 [Prashant Sharma] added eof new line ce8bf73 [Prashant Sharma] Added eof new line to all files. 4c35f2e [Prashant Sharma] SPARK-1563 Add package-info.java and package.scala files for all packages that appear in docs --- .../org/apache/spark/bagel/package-info.java | 21 ++++++++++++++ .../org/apache/spark/bagel/package.scala | 23 +++++++++++++++ .../spark/api/java/function/package-info.java | 23 +++++++++++++++ .../spark/api/java/function/package.scala | 25 +++++++++++++++++ .../apache/spark/annotation/package-info.java | 23 +++++++++++++++ .../org/apache/spark/annotation/package.scala | 25 +++++++++++++++++ .../apache/spark/api/java/package-info.java | 21 ++++++++++++++ .../apache/spark/broadcast/package-info.java | 21 ++++++++++++++ .../org/apache/spark/broadcast/package.scala | 2 +- .../apache/spark/executor/package-info.java | 21 ++++++++++++++ .../org/apache/spark/executor/package.scala | 24 ++++++++++++++++ .../org/apache/spark/io/package-info.java | 21 ++++++++++++++ .../scala/org/apache/spark/io/package.scala | 23 +++++++++++++++ .../apache/spark/metrics/sink/package.scala | 23 +++++++++++++++ .../apache/spark/metrics/source/package.scala | 23 +++++++++++++++ .../org/apache/spark/partial/package.scala | 28 +++++++++++++++++++ .../org/apache/spark/rdd/package-info.java | 21 ++++++++++++++ .../scala/org/apache/spark/rdd/package.scala | 23 +++++++++++++++ .../apache/spark/scheduler/package-info.java | 21 ++++++++++++++ .../org/apache/spark/scheduler/package.scala | 24 ++++++++++++++++ .../apache/spark/serializer/package-info.java | 21 ++++++++++++++ .../org/apache/spark/serializer/package.scala | 25 +++++++++++++++++ .../org/apache/spark/util/package-info.java | 21 ++++++++++++++ .../scala/org/apache/spark/util/package.scala | 23 +++++++++++++++ .../spark/util/random/package-info.java | 21 ++++++++++++++ .../apache/spark/util/random/package.scala | 23 +++++++++++++++ .../spark/streaming/flume/package-info.java | 21 ++++++++++++++ .../spark/streaming/flume/package.scala | 23 +++++++++++++++ .../spark/streaming/kafka/package-info.java | 21 ++++++++++++++ .../spark/streaming/kafka/package.scala | 23 +++++++++++++++ .../spark/streaming/mqtt/package-info.java | 21 ++++++++++++++ .../apache/spark/streaming/mqtt/package.scala | 23 +++++++++++++++ .../spark/streaming/twitter/package-info.java | 21 ++++++++++++++ .../spark/streaming/twitter/package.scala | 23 +++++++++++++++ .../spark/streaming/zeromq/package-info.java | 21 ++++++++++++++ .../spark/streaming/zeromq/package.scala | 23 +++++++++++++++ .../apache/spark/graphx/lib/package-info.java | 21 ++++++++++++++ .../org/apache/spark/graphx/lib/package.scala | 23 +++++++++++++++ .../org/apache/spark/graphx/package-info.java | 22 +++++++++++++++ .../spark/graphx/util/package-info.java | 21 ++++++++++++++ .../apache/spark/graphx/util/package.scala | 23 +++++++++++++++ .../spark/mllib/api/python/package.scala | 25 +++++++++++++++++ .../org/apache/spark/mllib/package-info.java | 21 ++++++++++++++ .../org/apache/spark/mllib/package.scala | 23 +++++++++++++++ .../org/apache/spark/sql/package-info.java | 21 ++++++++++++++ .../apache/spark/sql/hive/package-info.java | 18 ++++++++++++ .../org/apache/spark/sql/hive/package.scala | 20 +++++++++++++ .../streaming/api/java/package-info.java | 21 ++++++++++++++ .../spark/streaming/api/java/package.scala | 23 +++++++++++++++ .../spark/streaming/dstream/package-info.java | 21 ++++++++++++++ .../spark/streaming/dstream/package.scala | 24 ++++++++++++++++ 51 files changed, 1116 insertions(+), 1 deletion(-) create mode 100644 bagel/src/main/scala/org/apache/spark/bagel/package-info.java create mode 100644 bagel/src/main/scala/org/apache/spark/bagel/package.scala create mode 100644 core/src/main/java/org/apache/spark/api/java/function/package-info.java create mode 100644 core/src/main/java/org/apache/spark/api/java/function/package.scala create mode 100644 core/src/main/scala/org/apache/spark/annotation/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/annotation/package.scala create mode 100644 core/src/main/scala/org/apache/spark/api/java/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/broadcast/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/executor/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/executor/package.scala create mode 100644 core/src/main/scala/org/apache/spark/io/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/io/package.scala create mode 100644 core/src/main/scala/org/apache/spark/metrics/sink/package.scala create mode 100644 core/src/main/scala/org/apache/spark/metrics/source/package.scala create mode 100644 core/src/main/scala/org/apache/spark/partial/package.scala create mode 100644 core/src/main/scala/org/apache/spark/rdd/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/rdd/package.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/scheduler/package.scala create mode 100644 core/src/main/scala/org/apache/spark/serializer/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/serializer/package.scala create mode 100644 core/src/main/scala/org/apache/spark/util/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/util/package.scala create mode 100644 core/src/main/scala/org/apache/spark/util/random/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/util/random/package.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package-info.java create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala create mode 100644 external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package-info.java create mode 100644 external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala create mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package-info.java create mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala create mode 100644 external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package-info.java create mode 100644 external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/lib/package-info.java create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/package-info.java create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/util/package-info.java create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/util/package.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/api/python/package.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/package-info.java create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/package.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/package-info.java create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/package-info.java create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/api/java/package-info.java create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/api/java/package.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/dstream/package-info.java create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/dstream/package.scala diff --git a/bagel/src/main/scala/org/apache/spark/bagel/package-info.java b/bagel/src/main/scala/org/apache/spark/bagel/package-info.java new file mode 100644 index 0000000000000..81f26f276549f --- /dev/null +++ b/bagel/src/main/scala/org/apache/spark/bagel/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Bagel: An implementation of Pregel in Spark. THIS IS DEPRECATED - use Spark's GraphX library. + */ +package org.apache.spark.bagel; \ No newline at end of file diff --git a/bagel/src/main/scala/org/apache/spark/bagel/package.scala b/bagel/src/main/scala/org/apache/spark/bagel/package.scala new file mode 100644 index 0000000000000..2fb1934579781 --- /dev/null +++ b/bagel/src/main/scala/org/apache/spark/bagel/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Bagel: An implementation of Pregel in Spark. THIS IS DEPRECATED - use Spark's GraphX library. + */ +package object bagel diff --git a/core/src/main/java/org/apache/spark/api/java/function/package-info.java b/core/src/main/java/org/apache/spark/api/java/function/package-info.java new file mode 100644 index 0000000000000..463a42f23342c --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Set of interfaces to represent functions in Spark's Java API. Users create implementations of + * these interfaces to pass functions to various Java API methods for Spark. Please visit Spark's + * Java programming guide for more details. + */ +package org.apache.spark.api.java.function; \ No newline at end of file diff --git a/core/src/main/java/org/apache/spark/api/java/function/package.scala b/core/src/main/java/org/apache/spark/api/java/function/package.scala new file mode 100644 index 0000000000000..7f91de653a64a --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/package.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.java + +/** + * Set of interfaces to represent functions in Spark's Java API. Users create implementations of + * these interfaces to pass functions to various Java API methods for Spark. Please visit Spark's + * Java programming guide for more details. + */ +package object function \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/annotation/package-info.java b/core/src/main/scala/org/apache/spark/annotation/package-info.java new file mode 100644 index 0000000000000..12c7afe6f108a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark annotations to mark an API experimental or intended only for advanced usages by developers. + * This package consist of these annotations, which are used project wide and are reflected in + * Scala and Java docs. + */ +package org.apache.spark.annotation; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/annotation/package.scala b/core/src/main/scala/org/apache/spark/annotation/package.scala new file mode 100644 index 0000000000000..c3f4026a29d44 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/package.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Spark annotations to mark an API experimental or intended only for advanced usages by developers. + * This package consist of these annotations, which are used project wide and are reflected in + * Scala and Java docs. + */ +package object annotation diff --git a/core/src/main/scala/org/apache/spark/api/java/package-info.java b/core/src/main/scala/org/apache/spark/api/java/package-info.java new file mode 100644 index 0000000000000..10a480fc78e44 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/java/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark Java programming APIs. + */ +package org.apache.spark.api.java; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/broadcast/package-info.java b/core/src/main/scala/org/apache/spark/broadcast/package-info.java new file mode 100644 index 0000000000000..1510e6e84c7a8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/broadcast/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark's broadcast variables, used to broadcast immutable datasets to all nodes. + */ +package org.apache.spark.broadcast; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/broadcast/package.scala b/core/src/main/scala/org/apache/spark/broadcast/package.scala index 01bf88629a7dd..4f2c3c07161a3 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/package.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/package.scala @@ -18,7 +18,7 @@ package org.apache.spark /** - * Package for broadcast variables. See [[broadcast.Broadcast]] for details. + * Spark's broadcast variables, used to broadcast immutable datasets to all nodes. */ package object broadcast { // For package docs only diff --git a/core/src/main/scala/org/apache/spark/executor/package-info.java b/core/src/main/scala/org/apache/spark/executor/package-info.java new file mode 100644 index 0000000000000..dd3b6815fb45f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Package for executor components used with various cluster managers. + */ +package org.apache.spark.executor; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/executor/package.scala b/core/src/main/scala/org/apache/spark/executor/package.scala new file mode 100644 index 0000000000000..ef471d761d4b3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/package.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Executor components used with various cluster managers. + * See [[org.apache.spark.executor.Executor]]. + */ +package object executor diff --git a/core/src/main/scala/org/apache/spark/io/package-info.java b/core/src/main/scala/org/apache/spark/io/package-info.java new file mode 100644 index 0000000000000..bea1bfdb63751 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/io/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * IO codecs used for compression. + */ +package org.apache.spark.io; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/io/package.scala b/core/src/main/scala/org/apache/spark/io/package.scala new file mode 100644 index 0000000000000..f987e66a5dc44 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/io/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * IO codecs used for compression. See [[org.apache.spark.io.CompressionCodec]]. + */ +package object io diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/package.scala b/core/src/main/scala/org/apache/spark/metrics/sink/package.scala new file mode 100644 index 0000000000000..90e3aa70b99ef --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/sink/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.metrics + +/** + * Sinks used in Spark's metrics system. + */ +package object sink diff --git a/core/src/main/scala/org/apache/spark/metrics/source/package.scala b/core/src/main/scala/org/apache/spark/metrics/source/package.scala new file mode 100644 index 0000000000000..1e59fed6f7d7a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/source/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.metrics + +/** + * Sources used in Spark's metrics system. + */ +package object source diff --git a/core/src/main/scala/org/apache/spark/partial/package.scala b/core/src/main/scala/org/apache/spark/partial/package.scala new file mode 100644 index 0000000000000..62dc5cd25a164 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/partial/package.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * :: Experimental :: + * + * Support for approximate results. This provides convenient api and also implementation for + * approximate calculation. + * + * @see [[org.apache.spark.rdd.RDD.countApprox]] + */ +package object partial diff --git a/core/src/main/scala/org/apache/spark/rdd/package-info.java b/core/src/main/scala/org/apache/spark/rdd/package-info.java new file mode 100644 index 0000000000000..176cc58179fb0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Provides implementation's of various RDDs. + */ +package org.apache.spark.rdd; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/rdd/package.scala b/core/src/main/scala/org/apache/spark/rdd/package.scala new file mode 100644 index 0000000000000..55fc6e4d2b4df --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Provides several RDD implementations. See [[org.apache.spark.rdd.RDD]]. + */ +package object rdd diff --git a/core/src/main/scala/org/apache/spark/scheduler/package-info.java b/core/src/main/scala/org/apache/spark/scheduler/package-info.java new file mode 100644 index 0000000000000..5b4a628d3cee4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark's DAG scheduler. + */ +package org.apache.spark.scheduler; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/scheduler/package.scala b/core/src/main/scala/org/apache/spark/scheduler/package.scala new file mode 100644 index 0000000000000..f0dbfc2ac5f48 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/package.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Spark's scheduling components. This includes the [[org.apache.spark.scheduler.DAGScheduler]] and + * lower level [[org.apache.spark.scheduler.TaskScheduler]]. + */ +package object scheduler diff --git a/core/src/main/scala/org/apache/spark/serializer/package-info.java b/core/src/main/scala/org/apache/spark/serializer/package-info.java new file mode 100644 index 0000000000000..4c0b73ab36a00 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/serializer/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Pluggable serializers for RDD and shuffle data. + */ +package org.apache.spark.serializer; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/serializer/package.scala b/core/src/main/scala/org/apache/spark/serializer/package.scala new file mode 100644 index 0000000000000..8f09ef6f5b34c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/serializer/package.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Pluggable serializers for RDD and shuffle data. + * + * @see [[org.apache.spark.serializer.Serializer]] + */ +package object serializer diff --git a/core/src/main/scala/org/apache/spark/util/package-info.java b/core/src/main/scala/org/apache/spark/util/package-info.java new file mode 100644 index 0000000000000..819f54ee41a79 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark utilities. + */ +package org.apache.spark.util; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/util/package.scala b/core/src/main/scala/org/apache/spark/util/package.scala new file mode 100644 index 0000000000000..c7099b85d225f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Spark utilities. + */ +package object util diff --git a/core/src/main/scala/org/apache/spark/util/random/package-info.java b/core/src/main/scala/org/apache/spark/util/random/package-info.java new file mode 100644 index 0000000000000..62c3762dd11b6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/random/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Utilities for random number generation. + */ +package org.apache.spark.util.random; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/util/random/package.scala b/core/src/main/scala/org/apache/spark/util/random/package.scala new file mode 100644 index 0000000000000..f361125fe2e8a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/random/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +/** + * Utilities for random number generation. + */ +package object random diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java new file mode 100644 index 0000000000000..d31aa5f5c096c --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark streaming receiver for Flume. + */ +package org.apache.spark.streaming.flume; \ No newline at end of file diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala new file mode 100644 index 0000000000000..9bfab68c4b8b7 --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +/** + * Spark streaming receiver for Flume. + */ +package object flume diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package-info.java b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package-info.java new file mode 100644 index 0000000000000..947bae115a620 --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Kafka receiver for spark streaming. + */ +package org.apache.spark.streaming.kafka; \ No newline at end of file diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala new file mode 100644 index 0000000000000..47c5187f8751f --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +/** + * Kafka receiver for spark streaming, + */ +package object kafka diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package-info.java b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package-info.java new file mode 100644 index 0000000000000..728e0d8663d01 --- /dev/null +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * MQTT receiver for Spark Streaming. + */ +package org.apache.spark.streaming.mqtt; \ No newline at end of file diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala new file mode 100644 index 0000000000000..63d0d138183a9 --- /dev/null +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +/** + * MQTT receiver for Spark Streaming. + */ +package object mqtt diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package-info.java b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package-info.java new file mode 100644 index 0000000000000..258c0950a0aa7 --- /dev/null +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Twitter feed receiver for spark streaming. + */ +package org.apache.spark.streaming.twitter; \ No newline at end of file diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala new file mode 100644 index 0000000000000..580e37fa8f814 --- /dev/null +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +/** + * Twitter feed receiver for spark streaming. + */ +package object twitter diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package-info.java b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package-info.java new file mode 100644 index 0000000000000..587c524e2120f --- /dev/null +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Zeromq receiver for spark streaming. + */ +package org.apache.spark.streaming.zeromq; \ No newline at end of file diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala new file mode 100644 index 0000000000000..65e6e57f2c05d --- /dev/null +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +/** + * Zeromq receiver for spark streaming. + */ +package object zeromq diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/package-info.java b/graphx/src/main/scala/org/apache/spark/graphx/lib/package-info.java new file mode 100644 index 0000000000000..49ad7263141ce --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Internal support for MLLib Python API. + */ +package org.apache.spark.graphx.lib; \ No newline at end of file diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala new file mode 100644 index 0000000000000..7cfd617a32a34 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx + +/** + * Various analytics functions for graphs. + */ +package object lib diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package-info.java b/graphx/src/main/scala/org/apache/spark/graphx/package-info.java new file mode 100644 index 0000000000000..f659cc518ebd3 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * ALPHA COMPONENT + * GraphX is a graph processing framework built on top of Spark. + */ +package org.apache.spark.graphx; \ No newline at end of file diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/package-info.java b/graphx/src/main/scala/org/apache/spark/graphx/util/package-info.java new file mode 100644 index 0000000000000..90cd1d46db177 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Collections of utilities used by graphx. + */ +package org.apache.spark.graphx.util; \ No newline at end of file diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/package.scala new file mode 100644 index 0000000000000..2303541d2e427 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx + +/** + * Collections of utilities used by graphx. + */ +package object util diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/package.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/package.scala new file mode 100644 index 0000000000000..87bdc8558aaf5 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/package.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.api + +/** + * Internal support for MLLib Python API. + * + * @see [[org.apache.spark.mllib.api.python.PythonMLLibAPI]] + */ +package object python diff --git a/mllib/src/main/scala/org/apache/spark/mllib/package-info.java b/mllib/src/main/scala/org/apache/spark/mllib/package-info.java new file mode 100644 index 0000000000000..4991bc9e972c0 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark's machine learning library. + */ +package org.apache.spark.mllib; \ No newline at end of file diff --git a/mllib/src/main/scala/org/apache/spark/mllib/package.scala b/mllib/src/main/scala/org/apache/spark/mllib/package.scala new file mode 100644 index 0000000000000..5c2b2160c030e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Spark's machine learning library. + */ +package object mllib diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/package-info.java b/sql/catalyst/src/main/scala/org/apache/spark/sql/package-info.java new file mode 100644 index 0000000000000..53603614518f5 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Allows the execution of relational queries, including those expressed in SQL using Spark. + */ +package org.apache.spark.sql; \ No newline at end of file diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/package-info.java b/sql/hive/src/main/scala/org/apache/spark/sql/hive/package-info.java new file mode 100644 index 0000000000000..8b29fa7d1a8f7 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/package-info.java @@ -0,0 +1,18 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive; \ No newline at end of file diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala new file mode 100644 index 0000000000000..a6c8ed4f7e866 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +package object hive diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/package-info.java b/streaming/src/main/scala/org/apache/spark/streaming/api/java/package-info.java new file mode 100644 index 0000000000000..d43d949d76bb3 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Java APIs for spark streaming. + */ +package org.apache.spark.streaming.api.java; \ No newline at end of file diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/package.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/package.scala new file mode 100644 index 0000000000000..2479670f2a5bb --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.api + +/** + * Spark streaming's Java API. + */ +package object java diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/package-info.java b/streaming/src/main/scala/org/apache/spark/streaming/dstream/package-info.java new file mode 100644 index 0000000000000..05ca2ddffd3c0 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Various implementations of DStreams. + */ +package org.apache.spark.streaming.dstream; \ No newline at end of file diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/package.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/package.scala new file mode 100644 index 0000000000000..0179b92503e5d --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/package.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +/** + * Various implementations of DStream's. + * @see [[org.apache.spark.streaming.dstream.DStream]] + */ +package object dstream From bae07e36a6e0fb7982405316646b452b4ff06acc Mon Sep 17 00:00:00 2001 From: witgo Date: Wed, 14 May 2014 22:26:26 -0700 Subject: [PATCH 365/641] fix different versions of commons-lang dependency and apache/spark#746 addendum Author: witgo Closes #754 from witgo/commons-lang and squashes the following commits: 3ebab31 [witgo] merge master f3b8fa2 [witgo] merge master 2083fae [witgo] repeat definition 5599cdb [witgo] multiple version of sbt dependency c1b66a1 [witgo] fix different versions of commons-lang dependency --- project/SparkBuild.scala | 12 +++++++----- sql/hive/pom.xml | 8 ++++++++ 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 6adec556322ba..29dcd8678b476 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -318,6 +318,8 @@ object SparkBuild extends Build { val excludeFastutil = ExclusionRule(organization = "it.unimi.dsi") val excludeJruby = ExclusionRule(organization = "org.jruby") val excludeThrift = ExclusionRule(organization = "org.apache.thrift") + val excludeCommonsLang = ExclusionRule(organization = "commons-lang") + val excludeServletApi = ExclusionRule(organization = "javax.servlet", artifact = "servlet-api") def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark", version: String = "0.9.0-incubating", crossVersion: String = "2.10"): Option[sbt.ModuleID] = { @@ -491,7 +493,7 @@ object SparkBuild extends Build { javaOptions += "-XX:MaxPermSize=1g", libraryDependencies ++= Seq( "org.spark-project.hive" % "hive-metastore" % hiveVersion, - "org.spark-project.hive" % "hive-exec" % hiveVersion, + "org.spark-project.hive" % "hive-exec" % hiveVersion excludeAll(excludeCommonsLang, excludeCommonsLogging), "org.spark-project.hive" % "hive-serde" % hiveVersion ), // Multiple queries rely on the TestHive singleton. See comments there for more details. @@ -564,10 +566,10 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( // Exclude rule required for all ? "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-server-web-proxy" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm) + "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging), + "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging), + "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging), + "org.apache.hadoop" % "hadoop-yarn-server-web-proxy" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging, excludeServletApi) ) ) diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 9254b70e64a08..4d0b2fa1452a2 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -57,6 +57,10 @@ commons-logging commons-logging + + commons-lang + commons-lang + @@ -76,6 +80,10 @@ commons-logging commons-logging-api + + commons-lang + commons-lang + From 514157f2e4c341378f4a606b66562f39fe3eba5e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 14 May 2014 23:48:03 -0700 Subject: [PATCH 366/641] HOTFIX: Don't build Javadoc in Maven when creating releases. Because we've added java package descriptions in some packages that don't have any Java files, running the Javadoc target hits this issue: http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=4492654 To fix this I've simply removed the javadoc target when publishing releases. --- dev/create-release/create-release.sh | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index c4e74990860b6..c96afe822c6a9 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -49,16 +49,18 @@ if [[ ! "$@" =~ --package-only ]]; then mvn -Pyarn release:clean mvn -DskipTests \ - -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ + -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ + -Dmaven.javadoc.skip=true \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ --batch-mode release:prepare mvn -DskipTests \ - -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ + -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ + -Dmaven.javadoc.skip=true \ -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ release:perform From 3abe2b734a5578966f671c34f1de34b4446b90f1 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Thu, 15 May 2014 11:05:39 -0700 Subject: [PATCH 367/641] SPARK-1846 Ignore logs directory in RAT checks https://issues.apache.org/jira/browse/SPARK-1846 Author: Andrew Ash Closes #793 from ash211/SPARK-1846 and squashes the following commits: 3f50db5 [Andrew Ash] SPARK-1846 Ignore logs directory in RAT checks --- .rat-excludes | 1 + 1 file changed, 1 insertion(+) diff --git a/.rat-excludes b/.rat-excludes index 689467847d80d..15589702c5599 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -48,3 +48,4 @@ dist/* .*out .*ipr .*iws +logs From db8cc6f28abe4326cea6f53feb604920e4867a27 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 15 May 2014 11:20:21 -0700 Subject: [PATCH 368/641] [SPARK-1845] [SQL] Use AllScalaRegistrar for SparkSqlSerializer to register serializers of ... ...Scala collections. When I execute `orderBy` or `limit` for `SchemaRDD` including `ArrayType` or `MapType`, `SparkSqlSerializer` throws the following exception: ``` com.esotericsoftware.kryo.KryoException: Class cannot be created (missing no-arg constructor): scala.collection.immutable.$colon$colon ``` or ``` com.esotericsoftware.kryo.KryoException: Class cannot be created (missing no-arg constructor): scala.collection.immutable.Vector ``` or ``` com.esotericsoftware.kryo.KryoException: Class cannot be created (missing no-arg constructor): scala.collection.immutable.HashMap$HashTrieMap ``` and so on. This is because registrations of serializers for each concrete collections are missing in `SparkSqlSerializer`. I believe it should use `AllScalaRegistrar`. `AllScalaRegistrar` covers a lot of serializers for concrete classes of `Seq`, `Map` for `ArrayType`, `MapType`. Author: Takuya UESHIN Closes #790 from ueshin/issues/SPARK-1845 and squashes the following commits: d1ed992 [Takuya UESHIN] Use AllScalaRegistrar for SparkSqlSerializer to register serializers of Scala collections. --- .../sql/execution/SparkSqlSerializer.scala | 28 ++--------------- .../org/apache/spark/sql/DslQuerySuite.scala | 24 +++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 30 +++++++++++++++++++ .../scala/org/apache/spark/sql/TestData.scala | 10 +++++++ 4 files changed, 66 insertions(+), 26 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 94c2a249ef8f8..34b355e906695 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import com.clearspring.analytics.stream.cardinality.HyperLogLog import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Serializer, Kryo} +import com.twitter.chill.AllScalaRegistrar import org.apache.spark.{SparkEnv, SparkConf} import org.apache.spark.serializer.KryoSerializer @@ -35,22 +36,14 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co val kryo = new Kryo() kryo.setRegistrationRequired(false) kryo.register(classOf[MutablePair[_, _]]) - kryo.register(classOf[Array[Any]]) - // This is kinda hacky... - kryo.register(classOf[scala.collection.immutable.Map$Map1], new MapSerializer) - kryo.register(classOf[scala.collection.immutable.Map$Map2], new MapSerializer) - kryo.register(classOf[scala.collection.immutable.Map$Map3], new MapSerializer) - kryo.register(classOf[scala.collection.immutable.Map$Map4], new MapSerializer) - kryo.register(classOf[scala.collection.immutable.Map[_,_]], new MapSerializer) - kryo.register(classOf[scala.collection.Map[_,_]], new MapSerializer) kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericRow]) kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericMutableRow]) kryo.register(classOf[com.clearspring.analytics.stream.cardinality.HyperLogLog], new HyperLogLogSerializer) - kryo.register(classOf[scala.collection.mutable.ArrayBuffer[_]]) kryo.register(classOf[scala.math.BigDecimal], new BigDecimalSerializer) kryo.setReferences(false) kryo.setClassLoader(Utils.getSparkClassLoader) + new AllScalaRegistrar().apply(kryo) kryo } } @@ -97,20 +90,3 @@ private[sql] class HyperLogLogSerializer extends Serializer[HyperLogLog] { HyperLogLog.Builder.build(bytes) } } - -/** - * Maps do not have a no arg constructor and so cannot be serialized by default. So, we serialize - * them as `Array[(k,v)]`. - */ -private[sql] class MapSerializer extends Serializer[Map[_,_]] { - def write(kryo: Kryo, output: Output, map: Map[_,_]) { - kryo.writeObject(output, map.flatMap(e => Seq(e._1, e._2)).toArray) - } - - def read(kryo: Kryo, input: Input, tpe: Class[Map[_,_]]): Map[_,_] = { - kryo.readObject(input, classOf[Array[Any]]) - .sliding(2,2) - .map { case Array(k,v) => (k,v) } - .toMap - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 92a707ea57504..f43e98d614094 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -69,12 +69,36 @@ class DslQuerySuite extends QueryTest { checkAnswer( testData2.orderBy('a.desc, 'b.asc), Seq((3,1), (3,2), (2,1), (2,2), (1,1), (1,2))) + + checkAnswer( + arrayData.orderBy(GetItem('data, 0).asc), + arrayData.collect().sortBy(_.data(0)).toSeq) + + checkAnswer( + arrayData.orderBy(GetItem('data, 0).desc), + arrayData.collect().sortBy(_.data(0)).reverse.toSeq) + + checkAnswer( + mapData.orderBy(GetItem('data, 1).asc), + mapData.collect().sortBy(_.data(1)).toSeq) + + checkAnswer( + mapData.orderBy(GetItem('data, 1).desc), + mapData.collect().sortBy(_.data(1)).reverse.toSeq) } test("limit") { checkAnswer( testData.limit(10), testData.take(10).toSeq) + + checkAnswer( + arrayData.limit(1), + arrayData.take(1).toSeq) + + checkAnswer( + mapData.limit(1), + mapData.take(1).toSeq) } test("average") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 524549eb544fc..189dccd5253e5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -85,6 +85,36 @@ class SQLQuerySuite extends QueryTest { checkAnswer( sql("SELECT * FROM testData2 ORDER BY a DESC, b ASC"), Seq((3,1), (3,2), (2,1), (2,2), (1,1), (1,2))) + + checkAnswer( + sql("SELECT * FROM arrayData ORDER BY data[0] ASC"), + arrayData.collect().sortBy(_.data(0)).toSeq) + + checkAnswer( + sql("SELECT * FROM arrayData ORDER BY data[0] DESC"), + arrayData.collect().sortBy(_.data(0)).reverse.toSeq) + + checkAnswer( + sql("SELECT * FROM mapData ORDER BY data[1] ASC"), + mapData.collect().sortBy(_.data(1)).toSeq) + + checkAnswer( + sql("SELECT * FROM mapData ORDER BY data[1] DESC"), + mapData.collect().sortBy(_.data(1)).reverse.toSeq) + } + + test("limit") { + checkAnswer( + sql("SELECT * FROM testData LIMIT 10"), + testData.take(10).toSeq) + + checkAnswer( + sql("SELECT * FROM arrayData LIMIT 1"), + arrayData.collect().take(1).toSeq) + + checkAnswer( + sql("SELECT * FROM mapData LIMIT 1"), + mapData.collect().take(1).toSeq) } test("average") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index aa71e274f7f4b..1aca3872524d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -74,6 +74,16 @@ object TestData { ArrayData(Seq(2,3,4), Seq(Seq(2,3,4))) :: Nil) arrayData.registerAsTable("arrayData") + case class MapData(data: Map[Int, String]) + val mapData = + TestSQLContext.sparkContext.parallelize( + MapData(Map(1 -> "a1", 2 -> "b1", 3 -> "c1", 4 -> "d1", 5 -> "e1")) :: + MapData(Map(1 -> "a2", 2 -> "b2", 3 -> "c2", 4 -> "d2")) :: + MapData(Map(1 -> "a3", 2 -> "b3", 3 -> "c3")) :: + MapData(Map(1 -> "a4", 2 -> "b4")) :: + MapData(Map(1 -> "a5")) :: Nil) + mapData.registerAsTable("mapData") + case class StringData(s: String) val repeatedData = TestSQLContext.sparkContext.parallelize(List.fill(2)(StringData("test"))) From 94c9d6f59859ebc77fae112c2c42c64b7a4d7f83 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 15 May 2014 11:21:33 -0700 Subject: [PATCH 369/641] [SPARK-1819] [SQL] Fix GetField.nullable. `GetField.nullable` should be `true` not only when `field.nullable` is `true` but also when `child.nullable` is `true`. Author: Takuya UESHIN Closes #757 from ueshin/issues/SPARK-1819 and squashes the following commits: 8781a11 [Takuya UESHIN] Modify a test to use named parameters. 5bfc77d [Takuya UESHIN] Fix GetField.nullable. --- .../sql/catalyst/expressions/complexTypes.scala | 2 +- .../expressions/ExpressionEvaluationSuite.scala | 13 +++++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 195ca2eb3d589..b6aeae92f8bec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -74,7 +74,7 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio type EvaluatedType = Any def dataType = field.dataType - override def nullable = field.nullable + override def nullable = child.nullable || field.nullable override def foldable = child.foldable protected def structType = child.dataType match { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 344d8a304fc11..1132a30b42767 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -364,6 +364,19 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(GetField(BoundReference(2, AttributeReference("c", typeS)()), "a"), "aa", row) checkEvaluation(GetField(Literal(null, typeS), "a"), null, row) + + val typeS_notNullable = StructType( + StructField("a", StringType, nullable = false) + :: StructField("b", StringType, nullable = false) :: Nil + ) + + assert(GetField(BoundReference(2, + AttributeReference("c", typeS)()), "a").nullable === true) + assert(GetField(BoundReference(2, + AttributeReference("c", typeS_notNullable, nullable = false)()), "a").nullable === false) + + assert(GetField(Literal(null, typeS), "a").nullable === true) + assert(GetField(Literal(null, typeS_notNullable), "a").nullable === true) } test("arithmetic") { From d52761d67f42ad4d2ff02d96f0675fb3ab709f38 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 15 May 2014 11:59:59 -0700 Subject: [PATCH 370/641] [SPARK-1741][MLLIB] add predict(JavaRDD) to RegressionModel, ClassificationModel, and KMeans `model.predict` returns a RDD of Scala primitive type (Int/Double), which is recognized as Object in Java. Adding predict(JavaRDD) could make life easier for Java users. Added tests for KMeans, LinearRegression, and NaiveBayes. Will update examples after https://github.com/apache/spark/pull/653 gets merged. cc: @srowen Author: Xiangrui Meng Closes #670 from mengxr/predict-javardd and squashes the following commits: b77ccd8 [Xiangrui Meng] Merge branch 'master' into predict-javardd 43caac9 [Xiangrui Meng] add predict(JavaRDD) to RegressionModel, ClassificationModel, and KMeans --- .../classification/ClassificationModel.scala | 11 +++++++++- .../spark/mllib/clustering/KMeansModel.scala | 5 +++++ .../mllib/regression/RegressionModel.scala | 11 +++++++++- .../classification/JavaNaiveBayesSuite.java | 16 ++++++++++++++ .../mllib/clustering/JavaKMeansSuite.java | 14 +++++++++++++ .../regression/JavaLinearRegressionSuite.java | 21 +++++++++++++++++++ 6 files changed, 76 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala index 6332301e30cbd..b7a1d90d24d72 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala @@ -17,9 +17,10 @@ package org.apache.spark.mllib.classification +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD -import org.apache.spark.annotation.Experimental /** * :: Experimental :: @@ -43,4 +44,12 @@ trait ClassificationModel extends Serializable { * @return predicted category from the trained model */ def predict(testData: Vector): Double + + /** + * Predict values for examples stored in a JavaRDD. + * @param testData JavaRDD representing data points to be predicted + * @return a JavaRDD[java.lang.Double] where each entry contains the corresponding prediction + */ + def predict(testData: JavaRDD[Vector]): JavaRDD[java.lang.Double] = + predict(testData.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Double]] } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index ce14b06241932..fba21aefaaacd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.clustering +import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vector @@ -40,6 +41,10 @@ class KMeansModel private[mllib] (val clusterCenters: Array[Vector]) extends Ser points.map(p => KMeans.findClosest(centersWithNorm, new BreezeVectorWithNorm(p))._1) } + /** Maps given points to their cluster indices. */ + def predict(points: JavaRDD[Vector]): JavaRDD[java.lang.Integer] = + predict(points.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Integer]] + /** * Return the K-means cost (sum of squared distances of points to their nearest center) for this * model on the given data. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala index b27e158b43f9a..64b02f7a6e7a9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala @@ -17,9 +17,10 @@ package org.apache.spark.mllib.regression +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.annotation.Experimental @Experimental trait RegressionModel extends Serializable { @@ -38,4 +39,12 @@ trait RegressionModel extends Serializable { * @return Double prediction from the trained model */ def predict(testData: Vector): Double + + /** + * Predict values for examples stored in a JavaRDD. + * @param testData JavaRDD representing data points to be predicted + * @return a JavaRDD[java.lang.Double] where each entry contains the corresponding prediction + */ + def predict(testData: JavaRDD[Vector]): JavaRDD[java.lang.Double] = + predict(testData.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Double]] } diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java index c80b1134ed1b2..743a43a139c0c 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java @@ -19,6 +19,8 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; import org.junit.After; @@ -87,4 +89,18 @@ public void runUsingStaticMethods() { int numAccurate2 = validatePrediction(POINTS, model2); Assert.assertEquals(POINTS.size(), numAccurate2); } + + @Test + public void testPredictJavaRDD() { + JavaRDD examples = sc.parallelize(POINTS, 2).cache(); + NaiveBayesModel model = NaiveBayes.train(examples.rdd()); + JavaRDD vectors = examples.map(new Function() { + @Override + public Vector call(LabeledPoint v) throws Exception { + return v.features(); + }}); + JavaRDD predictions = model.predict(vectors); + // Should be able to get the first prediction. + predictions.first(); + } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java index 49a614bd90cab..0c916ca378034 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java @@ -88,4 +88,18 @@ public void runKMeansUsingConstructor() { .run(data.rdd()); assertEquals(expectedCenter, model.clusterCenters()[0]); } + + @Test + public void testPredictJavaRDD() { + List points = Lists.newArrayList( + Vectors.dense(1.0, 2.0, 6.0), + Vectors.dense(1.0, 3.0, 0.0), + Vectors.dense(1.0, 4.0, 6.0) + ); + JavaRDD data = sc.parallelize(points, 2); + KMeansModel model = new KMeans().setK(1).setMaxIterations(5).run(data.rdd()); + JavaRDD predictions = model.predict(data); + // Should be able to get the first prediction. + predictions.first(); + } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java index 7151e553512b3..6dc6877691036 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java @@ -25,8 +25,10 @@ import org.junit.Before; import org.junit.Test; +import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.util.LinearDataGenerator; public class JavaLinearRegressionSuite implements Serializable { @@ -92,4 +94,23 @@ public void runLinearRegressionUsingStaticMethods() { Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); } + @Test + public void testPredictJavaRDD() { + int nPoints = 100; + double A = 0.0; + double[] weights = {10, 10}; + JavaRDD testRDD = sc.parallelize( + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); + LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(); + LinearRegressionModel model = linSGDImpl.run(testRDD.rdd()); + JavaRDD vectors = testRDD.map(new Function() { + @Override + public Vector call(LabeledPoint v) throws Exception { + return v.features(); + } + }); + JavaRDD predictions = model.predict(vectors); + // Should be able to get the first prediction. + predictions.first(); + } } From 08e7606a964e3d1ac1d565f33651ff0035c75044 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 15 May 2014 16:35:39 -0700 Subject: [PATCH 371/641] SPARK-1851. Upgrade Avro dependency to 1.7.6 so Spark can read Avro file... ...s Author: Sandy Ryza Closes #795 from sryza/sandy-spark-1851 and squashes the following commits: 79c8227 [Sandy Ryza] SPARK-1851. Upgrade Avro dependency to 1.7.6 so Spark can read Avro files --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 786b6d4984d86..5df7036585b75 100644 --- a/pom.xml +++ b/pom.xml @@ -129,7 +129,7 @@ 8.1.14.v20131031 0.3.6 3.0.0 - 1.7.4 + 1.7.6 0.7.1 64m From e66e31be51f396c8f6b7a45119b8b31c4d8cdf79 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stevo=20Slavi=C4=87?= Date: Thu, 15 May 2014 16:44:14 -0700 Subject: [PATCH 372/641] SPARK-1803 Replaced colon in filenames with a dash MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This patch replaces colon in several filenames with dash to make these filenames Windows compatible. Author: Stevo Slavić Author: Stevo Slavic Closes #739 from sslavic/SPARK-1803 and squashes the following commits: 3ec66eb [Stevo Slavic] Removed extra empty line which was causing test to fail b967cc3 [Stevo Slavić] Aligned tests and names of test resources 2b12776 [Stevo Slavić] Fixed a typo in file name 1c5dfff [Stevo Slavić] Replaced colon in file name with dash 8f5bf7f [Stevo Slavić] Replaced colon in file name with dash c5b5083 [Stevo Slavić] Replaced colon in file name with dash a49801f [Stevo Slavić] Replaced colon in file name with dash 401d99e [Stevo Slavić] Replaced colon in file name with dash 40a9621 [Stevo Slavić] Replaced colon in file name with dash 4774580 [Stevo Slavić] Replaced colon in file name with dash 004f8bb [Stevo Slavić] Replaced colon in file name with dash d6a3e2c [Stevo Slavić] Replaced colon in file name with dash b585126 [Stevo Slavić] Replaced colon in file name with dash 028e48a [Stevo Slavić] Replaced colon in file name with dash ece0507 [Stevo Slavić] Replaced colon in file name with dash 84f5d2f [Stevo Slavić] Replaced colon in file name with dash 2fc7854 [Stevo Slavić] Replaced colon in file name with dash 9e1467d [Stevo Slavić] Replaced colon in file name with dash --- ...y test-0-cb0737d268260da971487a8ac5e3d8be} | 0 ...y test-0-c89dcddb8539bae211ea28cd1e695e35} | 0 ...y test-0-9589ac54d6de575083765ee64051ebaa} | 0 ...y test-0-d6d2726928df920ab99078015b8ff494} | 0 ...ry test-0-cfe78c68302bd143d989c383c3b0477} | 0 ...y test-0-358525951e893ed9686c46bf1f7447ee} | 0 ...y test-0-d9d50146537344742300ce7b52c18c91} | 0 ...y test-0-c25cdafe7d85c42a9d9cd559d88a6049} | 0 ...y test-0-63ab78355dbc6fa0d7619c8a35b0f7a7} | 0 ...y test-0-3adc3a7f76b2abd059904ba81a595db3} | 0 ...y test-0-681332efaff7b12e3ca12d9f021cd344} | 0 ...y test-0-3893ec7a86062215d9f817f1495a69bb} | 0 ...y test-0-75148567b91227053f728d72f7dc7c10} | 0 ...y test-0-4e4d2651e1e6e3c224242078d0201190} | 0 ...y test-0-b6e59464383591f02408f8765ac6a5d5} | 0 .../sql/hive/execution/PruningSuite.scala | 30 +++++++++---------- 16 files changed, 15 insertions(+), 15 deletions(-) rename sql/hive/src/test/resources/golden/{Column pruning: filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be => Column pruning - filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be} (100%) rename sql/hive/src/test/resources/golden/{Column pruning: non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 => Column pruning - non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35} (100%) rename sql/hive/src/test/resources/golden/{Column pruning: projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa => Column pruning - projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa} (100%) rename sql/hive/src/test/resources/golden/{Column pruning: simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 => Column pruning - simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494} (100%) rename sql/hive/src/test/resources/golden/{Column pruning: with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 => Column pruning - with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477} (100%) rename sql/hive/src/test/resources/golden/{Column pruning: with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee => Column pruning - with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee} (100%) rename sql/hive/src/test/resources/golden/{Column pruning: with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 => Column pruning - with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91} (100%) rename sql/hive/src/test/resources/golden/{Column pruning: without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 => Column pruning - without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049} (100%) rename sql/hive/src/test/resources/golden/{Partition pruning: all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 => Partition pruning - all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7} (100%) rename sql/hive/src/test/resources/golden/{Partition pruning: left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 => Partition pruning - left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3} (100%) rename sql/hive/src/test/resources/golden/{Partiton pruning: non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 => Partition pruning - non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344} (100%) rename sql/hive/src/test/resources/golden/{Partition pruning: non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb => Partition pruning - non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb} (100%) rename sql/hive/src/test/resources/golden/{Partition pruning: pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10 => Partition pruning - pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10} (100%) rename sql/hive/src/test/resources/golden/{Partition pruning: with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 => Partition pruning - with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190} (100%) rename sql/hive/src/test/resources/golden/{Partition pruning: with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 => Partition pruning - with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5} (100%) diff --git a/sql/hive/src/test/resources/golden/Column pruning: filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be b/sql/hive/src/test/resources/golden/Column pruning - filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be similarity index 100% rename from sql/hive/src/test/resources/golden/Column pruning: filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be rename to sql/hive/src/test/resources/golden/Column pruning - filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be diff --git a/sql/hive/src/test/resources/golden/Column pruning: non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 b/sql/hive/src/test/resources/golden/Column pruning - non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 similarity index 100% rename from sql/hive/src/test/resources/golden/Column pruning: non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 rename to sql/hive/src/test/resources/golden/Column pruning - non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 diff --git a/sql/hive/src/test/resources/golden/Column pruning: projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa b/sql/hive/src/test/resources/golden/Column pruning - projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa similarity index 100% rename from sql/hive/src/test/resources/golden/Column pruning: projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa rename to sql/hive/src/test/resources/golden/Column pruning - projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa diff --git a/sql/hive/src/test/resources/golden/Column pruning: simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 b/sql/hive/src/test/resources/golden/Column pruning - simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 similarity index 100% rename from sql/hive/src/test/resources/golden/Column pruning: simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 rename to sql/hive/src/test/resources/golden/Column pruning - simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 diff --git a/sql/hive/src/test/resources/golden/Column pruning: with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 b/sql/hive/src/test/resources/golden/Column pruning - with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 similarity index 100% rename from sql/hive/src/test/resources/golden/Column pruning: with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 rename to sql/hive/src/test/resources/golden/Column pruning - with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 diff --git a/sql/hive/src/test/resources/golden/Column pruning: with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee b/sql/hive/src/test/resources/golden/Column pruning - with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee similarity index 100% rename from sql/hive/src/test/resources/golden/Column pruning: with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee rename to sql/hive/src/test/resources/golden/Column pruning - with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee diff --git a/sql/hive/src/test/resources/golden/Column pruning: with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 b/sql/hive/src/test/resources/golden/Column pruning - with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 similarity index 100% rename from sql/hive/src/test/resources/golden/Column pruning: with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 rename to sql/hive/src/test/resources/golden/Column pruning - with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 diff --git a/sql/hive/src/test/resources/golden/Column pruning: without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 b/sql/hive/src/test/resources/golden/Column pruning - without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 similarity index 100% rename from sql/hive/src/test/resources/golden/Column pruning: without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 rename to sql/hive/src/test/resources/golden/Column pruning - without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 diff --git a/sql/hive/src/test/resources/golden/Partition pruning: all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 b/sql/hive/src/test/resources/golden/Partition pruning - all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 similarity index 100% rename from sql/hive/src/test/resources/golden/Partition pruning: all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 rename to sql/hive/src/test/resources/golden/Partition pruning - all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 diff --git a/sql/hive/src/test/resources/golden/Partition pruning: left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 b/sql/hive/src/test/resources/golden/Partition pruning - left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 similarity index 100% rename from sql/hive/src/test/resources/golden/Partition pruning: left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 rename to sql/hive/src/test/resources/golden/Partition pruning - left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 diff --git a/sql/hive/src/test/resources/golden/Partiton pruning: non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 b/sql/hive/src/test/resources/golden/Partition pruning - non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 similarity index 100% rename from sql/hive/src/test/resources/golden/Partiton pruning: non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 rename to sql/hive/src/test/resources/golden/Partition pruning - non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 diff --git a/sql/hive/src/test/resources/golden/Partition pruning: non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb b/sql/hive/src/test/resources/golden/Partition pruning - non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb similarity index 100% rename from sql/hive/src/test/resources/golden/Partition pruning: non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb rename to sql/hive/src/test/resources/golden/Partition pruning - non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb diff --git a/sql/hive/src/test/resources/golden/Partition pruning: pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10 b/sql/hive/src/test/resources/golden/Partition pruning - pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10 similarity index 100% rename from sql/hive/src/test/resources/golden/Partition pruning: pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10 rename to sql/hive/src/test/resources/golden/Partition pruning - pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10 diff --git a/sql/hive/src/test/resources/golden/Partition pruning: with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 b/sql/hive/src/test/resources/golden/Partition pruning - with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 similarity index 100% rename from sql/hive/src/test/resources/golden/Partition pruning: with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 rename to sql/hive/src/test/resources/golden/Partition pruning - with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 diff --git a/sql/hive/src/test/resources/golden/Partition pruning: with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 b/sql/hive/src/test/resources/golden/Partition pruning - with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 similarity index 100% rename from sql/hive/src/test/resources/golden/Partition pruning: with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 rename to sql/hive/src/test/resources/golden/Partition pruning - with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 25eca397462f7..34434449a0d77 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -28,7 +28,7 @@ import scala.collection.JavaConversions._ class PruningSuite extends HiveComparisonTest { // Column pruning tests - createPruningTest("Column pruning: with partitioned table", + createPruningTest("Column pruning - with partitioned table", "SELECT key FROM srcpart WHERE ds = '2008-04-08' LIMIT 3", Seq("key"), Seq("key"), @@ -36,43 +36,43 @@ class PruningSuite extends HiveComparisonTest { Seq("2008-04-08", "11"), Seq("2008-04-08", "12"))) - createPruningTest("Column pruning: with non-partitioned table", + createPruningTest("Column pruning - with non-partitioned table", "SELECT key FROM src WHERE key > 10 LIMIT 3", Seq("key"), Seq("key"), Seq.empty) - createPruningTest("Column pruning: with multiple projects", + createPruningTest("Column pruning - with multiple projects", "SELECT c1 FROM (SELECT key AS c1 FROM src WHERE key > 10) t1 LIMIT 3", Seq("c1"), Seq("key"), Seq.empty) - createPruningTest("Column pruning: projects alias substituting", + createPruningTest("Column pruning - projects alias substituting", "SELECT c1 AS c2 FROM (SELECT key AS c1 FROM src WHERE key > 10) t1 LIMIT 3", Seq("c2"), Seq("key"), Seq.empty) - createPruningTest("Column pruning: filter alias in-lining", + createPruningTest("Column pruning - filter alias in-lining", "SELECT c1 FROM (SELECT key AS c1 FROM src WHERE key > 10) t1 WHERE c1 < 100 LIMIT 3", Seq("c1"), Seq("key"), Seq.empty) - createPruningTest("Column pruning: without filters", + createPruningTest("Column pruning - without filters", "SELECT c1 FROM (SELECT key AS c1 FROM src) t1 LIMIT 3", Seq("c1"), Seq("key"), Seq.empty) - createPruningTest("Column pruning: simple top project without aliases", + createPruningTest("Column pruning - simple top project without aliases", "SELECT key FROM (SELECT key FROM src WHERE key > 10) t1 WHERE key < 100 LIMIT 3", Seq("key"), Seq("key"), Seq.empty) - createPruningTest("Column pruning: non-trivial top project with aliases", + createPruningTest("Column pruning - non-trivial top project with aliases", "SELECT c1 * 2 AS double FROM (SELECT key AS c1 FROM src WHERE key > 10) t1 LIMIT 3", Seq("double"), Seq("key"), @@ -80,19 +80,19 @@ class PruningSuite extends HiveComparisonTest { // Partition pruning tests - createPruningTest("Partition pruning: non-partitioned, non-trivial project", + createPruningTest("Partition pruning - non-partitioned, non-trivial project", "SELECT key * 2 AS double FROM src WHERE value IS NOT NULL", Seq("double"), Seq("key", "value"), Seq.empty) - createPruningTest("Partiton pruning: non-partitioned table", + createPruningTest("Partition pruning - non-partitioned table", "SELECT value FROM src WHERE key IS NOT NULL", Seq("value"), Seq("value", "key"), Seq.empty) - createPruningTest("Partition pruning: with filter on string partition key", + createPruningTest("Partition pruning - with filter on string partition key", "SELECT value, hr FROM srcpart1 WHERE ds = '2008-04-08'", Seq("value", "hr"), Seq("value", "hr"), @@ -100,7 +100,7 @@ class PruningSuite extends HiveComparisonTest { Seq("2008-04-08", "11"), Seq("2008-04-08", "12"))) - createPruningTest("Partition pruning: with filter on int partition key", + createPruningTest("Partition pruning - with filter on int partition key", "SELECT value, hr FROM srcpart1 WHERE hr < 12", Seq("value", "hr"), Seq("value", "hr"), @@ -108,20 +108,20 @@ class PruningSuite extends HiveComparisonTest { Seq("2008-04-08", "11"), Seq("2008-04-09", "11"))) - createPruningTest("Partition pruning: left only 1 partition", + createPruningTest("Partition pruning - left only 1 partition", "SELECT value, hr FROM srcpart1 WHERE ds = '2008-04-08' AND hr < 12", Seq("value", "hr"), Seq("value", "hr"), Seq( Seq("2008-04-08", "11"))) - createPruningTest("Partition pruning: all partitions pruned", + createPruningTest("Partition pruning - all partitions pruned", "SELECT value, hr FROM srcpart1 WHERE ds = '2014-01-27' AND hr = 11", Seq("value", "hr"), Seq("value", "hr"), Seq.empty) - createPruningTest("Partition pruning: pruning with both column key and partition key", + createPruningTest("Partition pruning - pruning with both column key and partition key", "SELECT value, hr FROM srcpart1 WHERE value IS NOT NULL AND hr < 12", Seq("value", "hr"), Seq("value", "hr"), From a4aafe5f9fb191533400caeafddf04986492c95f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 15 May 2014 16:50:42 -0700 Subject: [PATCH 373/641] [SQL] Fix tiny/small ints from HiveMetastore. Author: Michael Armbrust Closes #797 from marmbrus/smallInt and squashes the following commits: 2db9dae [Michael Armbrust] Fix tiny/small ints from HiveMetastore. --- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index fa30f596f8c67..636c4f7b93190 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -183,7 +183,8 @@ object HiveMetastoreTypes extends RegexParsers { "string" ^^^ StringType | "float" ^^^ FloatType | "int" ^^^ IntegerType | - "tinyint" ^^^ ShortType | + "tinyint" ^^^ ByteType | + "smallint" ^^^ ShortType | "double" ^^^ DoubleType | "bigint" ^^^ LongType | "binary" ^^^ BinaryType | @@ -227,7 +228,8 @@ object HiveMetastoreTypes extends RegexParsers { case StringType => "string" case FloatType => "float" case IntegerType => "int" - case ShortType =>"tinyint" + case ByteType => "tinyint" + case ShortType => "smallint" case DoubleType => "double" case LongType => "bigint" case BinaryType => "binary" From e1e3416c4e5f6f32983597d74866dbb809cf6a5e Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 15 May 2014 16:58:37 -0700 Subject: [PATCH 374/641] Fixes a misplaced comment. Fixes a misplaced comment from #785. @pwendell Author: Prashant Sharma Closes #788 from ScrapCodes/patch-1 and squashes the following commits: 3ef6a69 [Prashant Sharma] Update package-info.java 67d9461 [Prashant Sharma] Update package-info.java --- .../main/scala/org/apache/spark/graphx/lib/package-info.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/package-info.java b/graphx/src/main/scala/org/apache/spark/graphx/lib/package-info.java index 49ad7263141ce..e987ee8560ff7 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/package-info.java +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/package-info.java @@ -16,6 +16,6 @@ */ /** - * Internal support for MLLib Python API. + * Various analytics functions for graphs. */ -package org.apache.spark.graphx.lib; \ No newline at end of file +package org.apache.spark.graphx.lib; From 94c5139607ec876782e594012a108ebf55fa97db Mon Sep 17 00:00:00 2001 From: Huajian Mao Date: Thu, 15 May 2014 18:20:16 -0700 Subject: [PATCH 375/641] Typos in Spark Author: Huajian Mao Closes #798 from huajianmao/patch-1 and squashes the following commits: 208a454 [Huajian Mao] A typo in Task 1b515af [Huajian Mao] A typo in the message --- core/src/main/scala/org/apache/spark/scheduler/Task.scala | 2 +- .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 2ca3479c80efc..5871edeb856ad 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -33,7 +33,7 @@ import org.apache.spark.util.ByteBufferInputStream * - [[org.apache.spark.scheduler.ResultTask]] * * A Spark job consists of one or more stages. The very last stage in a job consists of multiple - * ResultTask's, while earlier stages consist of ShuffleMapTasks. A ResultTask executes the task + * ResultTasks, while earlier stages consist of ShuffleMapTasks. A ResultTask executes the task * and sends the task output back to the driver application. A ShuffleMapTask executes the task * and divides the task output to multiple buckets (based on the task's partitioner). * diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 649eed213e09a..17292b4c15b8b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -105,7 +105,7 @@ private[spark] class TaskSchedulerImpl( SchedulingMode.withName(schedulingModeConf.toUpperCase) } catch { case e: java.util.NoSuchElementException => - throw new SparkException(s"Urecognized spark.scheduler.mode: $schedulingModeConf") + throw new SparkException(s"Unrecognized spark.scheduler.mode: $schedulingModeConf") } // This is a var so that we can reset it for testing purposes. From bb98ecafce196ecc5bc3a1e4cc9264df7b752c6a Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 15 May 2014 21:37:58 -0700 Subject: [PATCH 376/641] SPARK-1860: Do not cleanup application work/ directories by default This causes an unrecoverable error for applications that are running for longer than 7 days that have jars added to the SparkContext, as the jars are cleaned up even though the application is still running. Author: Aaron Davidson Closes #800 from aarondav/shitty-defaults and squashes the following commits: a573fbb [Aaron Davidson] SPARK-1860: Do not cleanup application work/ directories by default --- .../main/scala/org/apache/spark/deploy/worker/Worker.scala | 2 +- docs/configuration.md | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 134624c35a57e..fb9cc116cd08b 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -65,7 +65,7 @@ private[spark] class Worker( val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 - val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", true) + val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) // How often worker will clean up old app folders val CLEANUP_INTERVAL_MILLIS = conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000 // TTL for app folders/data; after TTL expires it will be cleaned up diff --git a/docs/configuration.md b/docs/configuration.md index 2eed96f704a4f..f89040d714773 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -390,10 +390,11 @@ Apart from these, the following properties are also available, and may be useful spark.worker.cleanup.enabled - true + false Enable periodic cleanup of worker / application directories. Note that this only affects standalone - mode, as YARN works differently. + mode, as YARN works differently. Applications directories are cleaned up regardless of whether + the application is still running. From a20fea98811d98958567780815fcf0d4fb4e28d4 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 15 May 2014 22:12:34 -0700 Subject: [PATCH 377/641] [Spark-1461] Deferred Expression Evaluation (short-circuit evaluation) This patch unify the foldable & nullable interface for Expression. 1) Deterministic-less UDF (like Rand()) can not be folded. 2) Short-circut will significantly improves the performance in Expression Evaluation, however, the stateful UDF should not be ignored in a short-circuit evaluation(e.g. in expression: col1 > 0 and row_sequence() < 1000, row_sequence() can not be ignored even if col1 > 0 is false) I brought an concept of DeferredObject from Hive, which has 2 kinds of children classes (EagerResult / DeferredResult), the former requires triggering the evaluation before it's created, while the later trigger the evaluation when first called its get() method. Author: Cheng Hao Closes #446 from chenghao-intel/expression_deferred_evaluation and squashes the following commits: d2729de [Cheng Hao] Fix the codestyle issues a08f09c [Cheng Hao] fix bug in or/and short-circuit evaluation af2236b [Cheng Hao] revert the short-circuit expression evaluation for IF b7861d2 [Cheng Hao] Add Support for Deferred Expression Evaluation --- .../sql/catalyst/expressions/predicates.scala | 47 +++++++++++++------ .../org/apache/spark/sql/hive/hiveUdfs.scala | 28 ++++++++--- 2 files changed, 53 insertions(+), 22 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 6ee479939d25c..d111578530506 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -98,13 +98,19 @@ case class And(left: Expression, right: Expression) extends BinaryPredicate { override def eval(input: Row): Any = { val l = left.eval(input) - val r = right.eval(input) - if (l == false || r == false) { - false - } else if (l == null || r == null ) { - null + if (l == false) { + false } else { - true + val r = right.eval(input) + if (r == false) { + false + } else { + if (l != null && r != null) { + true + } else { + null + } + } } } } @@ -114,13 +120,19 @@ case class Or(left: Expression, right: Expression) extends BinaryPredicate { override def eval(input: Row): Any = { val l = left.eval(input) - val r = right.eval(input) - if (l == true || r == true) { + if (l == true) { true - } else if (l == null || r == null) { - null } else { - false + val r = right.eval(input) + if (r == true) { + true + } else { + if (l != null && r != null) { + false + } else { + null + } + } } } } @@ -133,8 +145,12 @@ case class Equals(left: Expression, right: Expression) extends BinaryComparison def symbol = "=" override def eval(input: Row): Any = { val l = left.eval(input) - val r = right.eval(input) - if (l == null || r == null) null else l == r + if (l == null) { + null + } else { + val r = right.eval(input) + if (r == null) null else l == r + } } } @@ -162,7 +178,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi extends Expression { def children = predicate :: trueValue :: falseValue :: Nil - def nullable = trueValue.nullable || falseValue.nullable + override def nullable = trueValue.nullable || falseValue.nullable def references = children.flatMap(_.references).toSet override lazy val resolved = childrenResolved && trueValue.dataType == falseValue.dataType def dataType = { @@ -175,8 +191,9 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi } type EvaluatedType = Any + override def eval(input: Row): Any = { - if (predicate.eval(input).asInstanceOf[Boolean]) { + if (true == predicate.eval(input)) { trueValue.eval(input) } else { falseValue.eval(input) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index d50e2c65b7b36..572902042337f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -248,17 +248,31 @@ private[hive] case class HiveGenericUdf(name: String, children: Seq[Expression]) isUDFDeterministic && children.foldLeft(true)((prev, n) => prev && n.foldable) } + protected lazy val deferedObjects = Array.fill[DeferredObject](children.length)({ + new DeferredObjectAdapter + }) + + // Adapter from Catalyst ExpressionResult to Hive DeferredObject + class DeferredObjectAdapter extends DeferredObject { + private var func: () => Any = _ + def set(func: () => Any) { + this.func = func + } + override def prepare(i: Int) = {} + override def get(): AnyRef = wrap(func()) + } + val dataType: DataType = inspectorToDataType(returnInspector) override def eval(input: Row): Any = { returnInspector // Make sure initialized. - val args = children.map { v => - new DeferredObject { - override def prepare(i: Int) = {} - override def get(): AnyRef = wrap(v.eval(input)) - } - }.toArray - unwrap(function.evaluate(args)) + var i = 0 + while (i < children.length) { + val idx = i + deferedObjects(i).asInstanceOf[DeferredObjectAdapter].set(() => {children(idx).eval(input)}) + i += 1 + } + unwrap(function.evaluate(deferedObjects)) } } From 17702e280c4b0b030870962fcb3d50c3085ae862 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 15 May 2014 23:31:43 -0700 Subject: [PATCH 378/641] SPARK-1862: Support for MapR in the Maven build. Author: Patrick Wendell Closes #803 from pwendell/mapr-support and squashes the following commits: 8df60e4 [Patrick Wendell] SPARK-1862: Support for MapR in the Maven build. --- pom.xml | 36 +++++++++++++++++++++++++++++++++++- 1 file changed, 35 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5df7036585b75..088d4cacb8bb5 100644 --- a/pom.xml +++ b/pom.xml @@ -123,6 +123,7 @@ 2.4.1 ${hadoop.version} 0.94.6 + 3.4.5 0.12.0 1.4.3 1.2.3 @@ -194,6 +195,17 @@ false + + mapr-repo + MapR Repository + http://repository.mapr.com/maven + + true + + + false + + @@ -490,6 +502,14 @@ commons-logging commons-logging + + org.mortbay.jetty + servlet-api-2.5 + + + junit + junit + @@ -979,6 +999,20 @@ + + mapr + + false + + + 1.0.3-mapr-3.0.3 + 2.3.0-mapr-4.0.0-beta + 0.94.17-mapr-1403 + 0.94.17-mapr-1403 + 3.4.5-mapr-1401 + + + hadoop-provided @@ -1024,7 +1058,7 @@ org.apache.zookeeper zookeeper - 3.4.5 + ${zookeeper.version} provided From e304eb998e3b2a44dc1f773cd75211679f217661 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 15 May 2014 23:33:27 -0700 Subject: [PATCH 379/641] HOTFIX: Duplication of hbase version --- pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/pom.xml b/pom.xml index 088d4cacb8bb5..86264d1132ec4 100644 --- a/pom.xml +++ b/pom.xml @@ -1008,7 +1008,6 @@ 1.0.3-mapr-3.0.3 2.3.0-mapr-4.0.0-beta 0.94.17-mapr-1403 - 0.94.17-mapr-1403 3.4.5-mapr-1401 From fa6de408a131a3e84350a60af74a92c323dfc5eb Mon Sep 17 00:00:00 2001 From: Zhen Peng Date: Fri, 16 May 2014 11:37:18 -0700 Subject: [PATCH 380/641] bugfix: overflow of graphx Edge compare function Author: Zhen Peng Closes #769 from zhpengg/bugfix-graphx-edge-compare and squashes the following commits: 8a978ff [Zhen Peng] add ut for graphx Edge.lexicographicOrdering.compare 413c258 [Zhen Peng] there maybe a overflow for two Long's substraction --- .../scala/org/apache/spark/graphx/Edge.scala | 10 ++++- .../org/apache/spark/graphx/EdgeSuite.scala | 39 +++++++++++++++++++ 2 files changed, 47 insertions(+), 2 deletions(-) create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/EdgeSuite.scala diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala index 580faa0866789..7e842ec4cc82f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala @@ -56,7 +56,13 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] object Edge { private[graphx] def lexicographicOrdering[ED] = new Ordering[Edge[ED]] { - override def compare(a: Edge[ED], b: Edge[ED]): Int = - (if (a.srcId != b.srcId) a.srcId - b.srcId else a.dstId - b.dstId).toInt + override def compare(a: Edge[ED], b: Edge[ED]): Int = { + if (a.srcId == b.srcId) { + if (a.dstId == b.dstId) 0 + else if (a.dstId < b.dstId) -1 + else 1 + } else if (a.srcId < b.srcId) -1 + else 1 + } } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/EdgeSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/EdgeSuite.scala new file mode 100644 index 0000000000000..5a2c73b414279 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/EdgeSuite.scala @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx + +import org.scalatest.FunSuite + +class EdgeSuite extends FunSuite { + test ("compare") { + // decending order + val testEdges: Array[Edge[Int]] = Array( + Edge(0x7FEDCBA987654321L, -0x7FEDCBA987654321L, 1), + Edge(0x2345L, 0x1234L, 1), + Edge(0x1234L, 0x5678L, 1), + Edge(0x1234L, 0x2345L, 1), + Edge(-0x7FEDCBA987654321L, 0x7FEDCBA987654321L, 1) + ) + // to ascending order + val sortedEdges = testEdges.sorted(Edge.lexicographicOrdering[Int]) + + for (i <- 0 until testEdges.length) { + assert(sortedEdges(i) == testEdges(testEdges.length - i - 1)) + } + } +} From 032d6632ad4ab88c97c9e568b63169a114220a02 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 16 May 2014 11:47:00 -0700 Subject: [PATCH 381/641] [SQL] Implement between in hql Author: Michael Armbrust Closes #804 from marmbrus/between and squashes the following commits: ae24672 [Michael Armbrust] add golden answer. d9997ef [Michael Armbrust] Implement between in hql. 9bd4433 [Michael Armbrust] Better error on parse failures. --- .../scala/org/apache/spark/sql/hive/HiveQl.scala | 16 ++++++++++++++++ .../between-0-df3cf89fcf2ef64199a582fae14a3321 | 1 + .../sql/hive/execution/HiveQuerySuite.scala | 4 ++++ 3 files changed, 21 insertions(+) create mode 100644 sql/hive/src/test/resources/golden/between-0-df3cf89fcf2ef64199a582fae14a3321 diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 1f688fe1117fe..93b9057a23816 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -233,6 +233,11 @@ private[hive] object HiveQl { } } catch { case e: Exception => throw new ParseException(sql, e) + case e: NotImplementedError => sys.error( + s""" + |Unsupported language features in query: $sql + |${dumpTree(getAst(sql))} + """.stripMargin) } } @@ -865,6 +870,17 @@ private[hive] object HiveQl { IsNull(nodeToExpr(child)) case Token("TOK_FUNCTION", Token("IN", Nil) :: value :: list) => In(nodeToExpr(value), list.map(nodeToExpr)) + case Token("TOK_FUNCTION", + Token("between", Nil) :: + Token("KW_FALSE", Nil) :: + target :: + minValue :: + maxValue :: Nil) => + + val targetExpression = nodeToExpr(target) + And( + GreaterThanOrEqual(targetExpression, nodeToExpr(minValue)), + LessThanOrEqual(targetExpression, nodeToExpr(maxValue))) /* Boolean Logic */ case Token(AND(), left :: right:: Nil) => And(nodeToExpr(left), nodeToExpr(right)) diff --git a/sql/hive/src/test/resources/golden/between-0-df3cf89fcf2ef64199a582fae14a3321 b/sql/hive/src/test/resources/golden/between-0-df3cf89fcf2ef64199a582fae14a3321 new file mode 100644 index 0000000000000..dcd1d8643e3cb --- /dev/null +++ b/sql/hive/src/test/resources/golden/between-0-df3cf89fcf2ef64199a582fae14a3321 @@ -0,0 +1 @@ +2 val_2 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 87a92d83383ab..1a2b2f89182ae 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -24,6 +24,10 @@ import org.apache.spark.sql.hive.test.TestHive._ */ class HiveQuerySuite extends HiveComparisonTest { + createQueryTest("between", + "SELECT * FROM src WHERE key between 1 and 2" + ) + test("Query expressed in SQL") { assert(sql("SELECT 1").collect() === Array(Seq(1))) } From 40d6acd6ba2feccc600301f5c47d4f90157138b1 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Fri, 16 May 2014 13:41:41 -0700 Subject: [PATCH 382/641] SPARK-1487 [SQL] Support record filtering via predicate pushdown in Parquet Simple filter predicates such as LessThan, GreaterThan, etc., where one side is a literal and the other one a NamedExpression are now pushed down to the underlying ParquetTableScan. Here are some results for a microbenchmark with a simple schema of six fields of different types where most records failed the test: | Uncompressed | Compressed -------------| ------------- | ------------- File size | 10 GB | 2 GB Speedup | 2 | 1.8 Since mileage may vary I added a new option to SparkConf: `org.apache.spark.sql.parquet.filter.pushdown` Default value would be `true` and setting it to `false` disables the pushdown. When most rows are expected to pass the filter or when there are few fields performance can be better when pushdown is disabled. The default should fit situations with a reasonable number of (possibly nested) fields where not too many records on average pass the filter. Because of an issue with Parquet ([see here](https://github.com/Parquet/parquet-mr/issues/371])) currently only predicates on non-nullable attributes are pushed down. If one would know that for a given table no optional fields have missing values one could also allow overriding this. Author: Andre Schumacher Closes #511 from AndreSchumacher/parquet_filter and squashes the following commits: 16bfe83 [Andre Schumacher] Removing leftovers from merge during rebase 7b304ca [Andre Schumacher] Fixing formatting c36d5cb [Andre Schumacher] Scalastyle 3da98db [Andre Schumacher] Second round of review feedback 7a78265 [Andre Schumacher] Fixing broken formatting in ParquetFilter a86553b [Andre Schumacher] First round of code review feedback b0f7806 [Andre Schumacher] Optimizing imports in ParquetTestData 85fea2d [Andre Schumacher] Adding SparkConf setting to disable filter predicate pushdown f0ad3cf [Andre Schumacher] Undoing changes not needed for this PR 210e9cb [Andre Schumacher] Adding disjunctive filter predicates a93a588 [Andre Schumacher] Adding unit test for filtering 6d22666 [Andre Schumacher] Extending ParquetFilters 93e8192 [Andre Schumacher] First commit Parquet record filtering --- .../spark/sql/execution/SparkStrategies.scala | 31 +- .../spark/sql/parquet/ParquetFilters.scala | 436 ++++++++++++++++++ .../sql/parquet/ParquetTableOperations.scala | 90 +++- .../spark/sql/parquet/ParquetTestData.scala | 90 +++- .../spark/sql/parquet/ParquetQuerySuite.scala | 135 +++++- 5 files changed, 731 insertions(+), 51 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index f763106da4e0e..394a59700dbaf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -140,12 +140,35 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { InsertIntoParquetTable(relation, planLater(child), overwrite=true)(sparkContext) :: Nil case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => InsertIntoParquetTable(table, planLater(child), overwrite)(sparkContext) :: Nil - case PhysicalOperation(projectList, filters, relation: ParquetRelation) => - // TODO: Should be pushing down filters as well. + case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) => { + val remainingFilters = + if (sparkContext.conf.getBoolean(ParquetFilters.PARQUET_FILTER_PUSHDOWN_ENABLED, true)) { + filters.filter { + // Note: filters cannot be pushed down to Parquet if they contain more complex + // expressions than simple "Attribute cmp Literal" comparisons. Here we remove + // all filters that have been pushed down. Note that a predicate such as + // "(A AND B) OR C" can result in "A OR C" being pushed down. + filter => + val recordFilter = ParquetFilters.createFilter(filter) + if (!recordFilter.isDefined) { + // First case: the pushdown did not result in any record filter. + true + } else { + // Second case: a record filter was created; here we are conservative in + // the sense that even if "A" was pushed and we check for "A AND B" we + // still want to keep "A AND B" in the higher-level filter, not just "B". + !ParquetFilters.findExpression(recordFilter.get, filter).isDefined + } + } + } else { + filters + } pruneFilterProject( projectList, - filters, - ParquetTableScan(_, relation, None)(sparkContext)) :: Nil + remainingFilters, + ParquetTableScan(_, relation, filters)(sparkContext)) :: Nil + } + case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala new file mode 100644 index 0000000000000..052b0a9196717 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -0,0 +1,436 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.parquet + +import org.apache.hadoop.conf.Configuration + +import parquet.filter._ +import parquet.filter.ColumnPredicates._ +import parquet.column.ColumnReader + +import com.google.common.io.BaseEncoding + +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.expressions.{Predicate => CatalystPredicate} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.SparkSqlSerializer + +object ParquetFilters { + val PARQUET_FILTER_DATA = "org.apache.spark.sql.parquet.row.filter" + // set this to false if pushdown should be disabled + val PARQUET_FILTER_PUSHDOWN_ENABLED = "spark.sql.hints.parquetFilterPushdown" + + def createRecordFilter(filterExpressions: Seq[Expression]): UnboundRecordFilter = { + val filters: Seq[CatalystFilter] = filterExpressions.collect { + case (expression: Expression) if createFilter(expression).isDefined => + createFilter(expression).get + } + if (filters.length > 0) filters.reduce(AndRecordFilter.and) else null + } + + def createFilter(expression: Expression): Option[CatalystFilter] = { + def createEqualityFilter( + name: String, + literal: Literal, + predicate: CatalystPredicate) = literal.dataType match { + case BooleanType => + ComparisonFilter.createBooleanFilter(name, literal.value.asInstanceOf[Boolean], predicate) + case IntegerType => + ComparisonFilter.createIntFilter( + name, + (x: Int) => x == literal.value.asInstanceOf[Int], + predicate) + case LongType => + ComparisonFilter.createLongFilter( + name, + (x: Long) => x == literal.value.asInstanceOf[Long], + predicate) + case DoubleType => + ComparisonFilter.createDoubleFilter( + name, + (x: Double) => x == literal.value.asInstanceOf[Double], + predicate) + case FloatType => + ComparisonFilter.createFloatFilter( + name, + (x: Float) => x == literal.value.asInstanceOf[Float], + predicate) + case StringType => + ComparisonFilter.createStringFilter(name, literal.value.asInstanceOf[String], predicate) + } + def createLessThanFilter( + name: String, + literal: Literal, + predicate: CatalystPredicate) = literal.dataType match { + case IntegerType => + ComparisonFilter.createIntFilter( + name, + (x: Int) => x < literal.value.asInstanceOf[Int], + predicate) + case LongType => + ComparisonFilter.createLongFilter( + name, + (x: Long) => x < literal.value.asInstanceOf[Long], + predicate) + case DoubleType => + ComparisonFilter.createDoubleFilter( + name, + (x: Double) => x < literal.value.asInstanceOf[Double], + predicate) + case FloatType => + ComparisonFilter.createFloatFilter( + name, + (x: Float) => x < literal.value.asInstanceOf[Float], + predicate) + } + def createLessThanOrEqualFilter( + name: String, + literal: Literal, + predicate: CatalystPredicate) = literal.dataType match { + case IntegerType => + ComparisonFilter.createIntFilter( + name, + (x: Int) => x <= literal.value.asInstanceOf[Int], + predicate) + case LongType => + ComparisonFilter.createLongFilter( + name, + (x: Long) => x <= literal.value.asInstanceOf[Long], + predicate) + case DoubleType => + ComparisonFilter.createDoubleFilter( + name, + (x: Double) => x <= literal.value.asInstanceOf[Double], + predicate) + case FloatType => + ComparisonFilter.createFloatFilter( + name, + (x: Float) => x <= literal.value.asInstanceOf[Float], + predicate) + } + // TODO: combine these two types somehow? + def createGreaterThanFilter( + name: String, + literal: Literal, + predicate: CatalystPredicate) = literal.dataType match { + case IntegerType => + ComparisonFilter.createIntFilter( + name, + (x: Int) => x > literal.value.asInstanceOf[Int], + predicate) + case LongType => + ComparisonFilter.createLongFilter( + name, + (x: Long) => x > literal.value.asInstanceOf[Long], + predicate) + case DoubleType => + ComparisonFilter.createDoubleFilter( + name, + (x: Double) => x > literal.value.asInstanceOf[Double], + predicate) + case FloatType => + ComparisonFilter.createFloatFilter( + name, + (x: Float) => x > literal.value.asInstanceOf[Float], + predicate) + } + def createGreaterThanOrEqualFilter( + name: String, + literal: Literal, + predicate: CatalystPredicate) = literal.dataType match { + case IntegerType => + ComparisonFilter.createIntFilter( + name, (x: Int) => x >= literal.value.asInstanceOf[Int], + predicate) + case LongType => + ComparisonFilter.createLongFilter( + name, + (x: Long) => x >= literal.value.asInstanceOf[Long], + predicate) + case DoubleType => + ComparisonFilter.createDoubleFilter( + name, + (x: Double) => x >= literal.value.asInstanceOf[Double], + predicate) + case FloatType => + ComparisonFilter.createFloatFilter( + name, + (x: Float) => x >= literal.value.asInstanceOf[Float], + predicate) + } + + /** + * TODO: we currently only filter on non-nullable (Parquet REQUIRED) attributes until + * https://github.com/Parquet/parquet-mr/issues/371 + * has been resolved. + */ + expression match { + case p @ Or(left: Expression, right: Expression) + if createFilter(left).isDefined && createFilter(right).isDefined => { + // If either side of this Or-predicate is empty then this means + // it contains a more complex comparison than between attribute and literal + // (e.g., it contained a CAST). The only safe thing to do is then to disregard + // this disjunction, which could be contained in a conjunction. If it stands + // alone then it is also safe to drop it, since a Null return value of this + // function is interpreted as having no filters at all. + val leftFilter = createFilter(left).get + val rightFilter = createFilter(right).get + Some(new OrFilter(leftFilter, rightFilter)) + } + case p @ And(left: Expression, right: Expression) => { + // This treats nested conjunctions; since either side of the conjunction + // may contain more complex filter expressions we may actually generate + // strictly weaker filter predicates in the process. + val leftFilter = createFilter(left) + val rightFilter = createFilter(right) + (leftFilter, rightFilter) match { + case (None, Some(filter)) => Some(filter) + case (Some(filter), None) => Some(filter) + case (_, _) => + Some(new AndFilter(leftFilter.get, rightFilter.get)) + } + } + case p @ Equals(left: Literal, right: NamedExpression) if !right.nullable => + Some(createEqualityFilter(right.name, left, p)) + case p @ Equals(left: NamedExpression, right: Literal) if !left.nullable => + Some(createEqualityFilter(left.name, right, p)) + case p @ LessThan(left: Literal, right: NamedExpression) if !right.nullable => + Some(createLessThanFilter(right.name, left, p)) + case p @ LessThan(left: NamedExpression, right: Literal) if !left.nullable => + Some(createLessThanFilter(left.name, right, p)) + case p @ LessThanOrEqual(left: Literal, right: NamedExpression) if !right.nullable => + Some(createLessThanOrEqualFilter(right.name, left, p)) + case p @ LessThanOrEqual(left: NamedExpression, right: Literal) if !left.nullable => + Some(createLessThanOrEqualFilter(left.name, right, p)) + case p @ GreaterThan(left: Literal, right: NamedExpression) if !right.nullable => + Some(createGreaterThanFilter(right.name, left, p)) + case p @ GreaterThan(left: NamedExpression, right: Literal) if !left.nullable => + Some(createGreaterThanFilter(left.name, right, p)) + case p @ GreaterThanOrEqual(left: Literal, right: NamedExpression) if !right.nullable => + Some(createGreaterThanOrEqualFilter(right.name, left, p)) + case p @ GreaterThanOrEqual(left: NamedExpression, right: Literal) if !left.nullable => + Some(createGreaterThanOrEqualFilter(left.name, right, p)) + case _ => None + } + } + + /** + * Note: Inside the Hadoop API we only have access to `Configuration`, not to + * [[org.apache.spark.SparkContext]], so we cannot use broadcasts to convey + * the actual filter predicate. + */ + def serializeFilterExpressions(filters: Seq[Expression], conf: Configuration): Unit = { + if (filters.length > 0) { + val serialized: Array[Byte] = SparkSqlSerializer.serialize(filters) + val encoded: String = BaseEncoding.base64().encode(serialized) + conf.set(PARQUET_FILTER_DATA, encoded) + } + } + + /** + * Note: Inside the Hadoop API we only have access to `Configuration`, not to + * [[org.apache.spark.SparkContext]], so we cannot use broadcasts to convey + * the actual filter predicate. + */ + def deserializeFilterExpressions(conf: Configuration): Seq[Expression] = { + val data = conf.get(PARQUET_FILTER_DATA) + if (data != null) { + val decoded: Array[Byte] = BaseEncoding.base64().decode(data) + SparkSqlSerializer.deserialize(decoded) + } else { + Seq() + } + } + + /** + * Try to find the given expression in the tree of filters in order to + * determine whether it is safe to remove it from the higher level filters. Note + * that strictly speaking we could stop the search whenever an expression is found + * that contains this expression as subexpression (e.g., when searching for "a" + * and "(a or c)" is found) but we don't care about optimizations here since the + * filter tree is assumed to be small. + * + * @param filter The [[org.apache.spark.sql.parquet.CatalystFilter]] to expand + * and search + * @param expression The expression to look for + * @return An optional [[org.apache.spark.sql.parquet.CatalystFilter]] that + * contains the expression. + */ + def findExpression( + filter: CatalystFilter, + expression: Expression): Option[CatalystFilter] = filter match { + case f @ OrFilter(_, leftFilter, rightFilter, _) => + if (f.predicate == expression) { + Some(f) + } else { + val left = findExpression(leftFilter, expression) + if (left.isDefined) left else findExpression(rightFilter, expression) + } + case f @ AndFilter(_, leftFilter, rightFilter, _) => + if (f.predicate == expression) { + Some(f) + } else { + val left = findExpression(leftFilter, expression) + if (left.isDefined) left else findExpression(rightFilter, expression) + } + case f @ ComparisonFilter(_, _, predicate) => + if (predicate == expression) Some(f) else None + case _ => None + } +} + +abstract private[parquet] class CatalystFilter( + @transient val predicate: CatalystPredicate) extends UnboundRecordFilter + +private[parquet] case class ComparisonFilter( + val columnName: String, + private var filter: UnboundRecordFilter, + @transient override val predicate: CatalystPredicate) + extends CatalystFilter(predicate) { + override def bind(readers: java.lang.Iterable[ColumnReader]): RecordFilter = { + filter.bind(readers) + } +} + +private[parquet] case class OrFilter( + private var filter: UnboundRecordFilter, + @transient val left: CatalystFilter, + @transient val right: CatalystFilter, + @transient override val predicate: Or) + extends CatalystFilter(predicate) { + def this(l: CatalystFilter, r: CatalystFilter) = + this( + OrRecordFilter.or(l, r), + l, + r, + Or(l.predicate, r.predicate)) + + override def bind(readers: java.lang.Iterable[ColumnReader]): RecordFilter = { + filter.bind(readers) + } +} + +private[parquet] case class AndFilter( + private var filter: UnboundRecordFilter, + @transient val left: CatalystFilter, + @transient val right: CatalystFilter, + @transient override val predicate: And) + extends CatalystFilter(predicate) { + def this(l: CatalystFilter, r: CatalystFilter) = + this( + AndRecordFilter.and(l, r), + l, + r, + And(l.predicate, r.predicate)) + + override def bind(readers: java.lang.Iterable[ColumnReader]): RecordFilter = { + filter.bind(readers) + } +} + +private[parquet] object ComparisonFilter { + def createBooleanFilter( + columnName: String, + value: Boolean, + predicate: CatalystPredicate): CatalystFilter = + new ComparisonFilter( + columnName, + ColumnRecordFilter.column( + columnName, + ColumnPredicates.applyFunctionToBoolean( + new BooleanPredicateFunction { + def functionToApply(input: Boolean): Boolean = input == value + } + )), + predicate) + + def createStringFilter( + columnName: String, + value: String, + predicate: CatalystPredicate): CatalystFilter = + new ComparisonFilter( + columnName, + ColumnRecordFilter.column( + columnName, + ColumnPredicates.applyFunctionToString ( + new ColumnPredicates.PredicateFunction[String] { + def functionToApply(input: String): Boolean = input == value + } + )), + predicate) + + def createIntFilter( + columnName: String, + func: Int => Boolean, + predicate: CatalystPredicate): CatalystFilter = + new ComparisonFilter( + columnName, + ColumnRecordFilter.column( + columnName, + ColumnPredicates.applyFunctionToInteger( + new IntegerPredicateFunction { + def functionToApply(input: Int) = func(input) + } + )), + predicate) + + def createLongFilter( + columnName: String, + func: Long => Boolean, + predicate: CatalystPredicate): CatalystFilter = + new ComparisonFilter( + columnName, + ColumnRecordFilter.column( + columnName, + ColumnPredicates.applyFunctionToLong( + new LongPredicateFunction { + def functionToApply(input: Long) = func(input) + } + )), + predicate) + + def createDoubleFilter( + columnName: String, + func: Double => Boolean, + predicate: CatalystPredicate): CatalystFilter = + new ComparisonFilter( + columnName, + ColumnRecordFilter.column( + columnName, + ColumnPredicates.applyFunctionToDouble( + new DoublePredicateFunction { + def functionToApply(input: Double) = func(input) + } + )), + predicate) + + def createFloatFilter( + columnName: String, + func: Float => Boolean, + predicate: CatalystPredicate): CatalystFilter = + new ComparisonFilter( + columnName, + ColumnRecordFilter.column( + columnName, + ColumnPredicates.applyFunctionToFloat( + new FloatPredicateFunction { + def functionToApply(input: Float) = func(input) + } + )), + predicate) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index f825ca3c028ef..65ba1246fbf9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -27,26 +27,27 @@ import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat, FileOutputCommitter} -import parquet.hadoop.{ParquetInputFormat, ParquetOutputFormat} +import parquet.hadoop.{ParquetRecordReader, ParquetInputFormat, ParquetOutputFormat} +import parquet.hadoop.api.ReadSupport import parquet.hadoop.util.ContextUtil import parquet.io.InvalidRecordException import parquet.schema.MessageType -import org.apache.spark.{SerializableWritable, SparkContext, TaskContext} +import org.apache.spark.{Logging, SerializableWritable, SparkContext, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row} import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode} /** * Parquet table scan operator. Imports the file that backs the given - * [[ParquetRelation]] as a RDD[Row]. + * [[org.apache.spark.sql.parquet.ParquetRelation]] as a ``RDD[Row]``. */ case class ParquetTableScan( // note: output cannot be transient, see // https://issues.apache.org/jira/browse/SPARK-1367 output: Seq[Attribute], relation: ParquetRelation, - columnPruningPred: Option[Expression])( + columnPruningPred: Seq[Expression])( @transient val sc: SparkContext) extends LeafNode { @@ -62,18 +63,30 @@ case class ParquetTableScan( for (path <- fileList if !path.getName.startsWith("_")) { NewFileInputFormat.addInputPath(job, path) } + + // Store Parquet schema in `Configuration` conf.set( RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, ParquetTypesConverter.convertFromAttributes(output).toString) - // TODO: think about adding record filters - /* Comments regarding record filters: it would be nice to push down as much filtering - to Parquet as possible. However, currently it seems we cannot pass enough information - to materialize an (arbitrary) Catalyst [[Predicate]] inside Parquet's - ``FilteredRecordReader`` (via Configuration, for example). Simple - filter-rows-by-column-values however should be supported. - */ - sc.newAPIHadoopRDD(conf, classOf[ParquetInputFormat[Row]], classOf[Void], classOf[Row]) - .map(_._2) + + // Store record filtering predicate in `Configuration` + // Note 1: the input format ignores all predicates that cannot be expressed + // as simple column predicate filters in Parquet. Here we just record + // the whole pruning predicate. + // Note 2: you can disable filter predicate pushdown by setting + // "spark.sql.hints.parquetFilterPushdown" to false inside SparkConf. + if (columnPruningPred.length > 0 && + sc.conf.getBoolean(ParquetFilters.PARQUET_FILTER_PUSHDOWN_ENABLED, true)) { + ParquetFilters.serializeFilterExpressions(columnPruningPred, conf) + } + + sc.newAPIHadoopRDD( + conf, + classOf[org.apache.spark.sql.parquet.FilteringParquetRowInputFormat], + classOf[Void], + classOf[Row]) + .map(_._2) + .filter(_ != null) // Parquet's record filters may produce null values } override def otherCopyArgs = sc :: Nil @@ -184,10 +197,19 @@ case class InsertIntoParquetTable( override def otherCopyArgs = sc :: Nil - // based on ``saveAsNewAPIHadoopFile`` in [[PairRDDFunctions]] - // TODO: Maybe PairRDDFunctions should use Product2 instead of Tuple2? - // .. then we could use the default one and could use [[MutablePair]] - // instead of ``Tuple2`` + /** + * Stores the given Row RDD as a Hadoop file. + * + * Note: We cannot use ``saveAsNewAPIHadoopFile`` from [[org.apache.spark.rdd.PairRDDFunctions]] + * together with [[org.apache.spark.util.MutablePair]] because ``PairRDDFunctions`` uses + * ``Tuple2`` and not ``Product2``. Also, we want to allow appending files to an existing + * directory and need to determine which was the largest written file index before starting to + * write. + * + * @param rdd The [[org.apache.spark.rdd.RDD]] to writer + * @param path The directory to write to. + * @param conf A [[org.apache.hadoop.conf.Configuration]]. + */ private def saveAsHadoopFile( rdd: RDD[Row], path: String, @@ -244,8 +266,10 @@ case class InsertIntoParquetTable( } } -// TODO: this will be able to append to directories it created itself, not necessarily -// to imported ones +/** + * TODO: this will be able to append to directories it created itself, not necessarily + * to imported ones. + */ private[parquet] class AppendingParquetOutputFormat(offset: Int) extends parquet.hadoop.ParquetOutputFormat[Row] { // override to accept existing directories as valid output directory @@ -262,6 +286,30 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int) } } +/** + * We extend ParquetInputFormat in order to have more control over which + * RecordFilter we want to use. + */ +private[parquet] class FilteringParquetRowInputFormat + extends parquet.hadoop.ParquetInputFormat[Row] with Logging { + override def createRecordReader( + inputSplit: InputSplit, + taskAttemptContext: TaskAttemptContext): RecordReader[Void, Row] = { + val readSupport: ReadSupport[Row] = new RowReadSupport() + + val filterExpressions = + ParquetFilters.deserializeFilterExpressions(ContextUtil.getConfiguration(taskAttemptContext)) + if (filterExpressions.length > 0) { + logInfo(s"Pushing down predicates for RecordFilter: ${filterExpressions.mkString(", ")}") + new ParquetRecordReader[Row]( + readSupport, + ParquetFilters.createRecordFilter(filterExpressions)) + } else { + new ParquetRecordReader[Row](readSupport) + } + } +} + private[parquet] object FileSystemHelper { def listFiles(pathStr: String, conf: Configuration): Seq[Path] = { val origPath = new Path(pathStr) @@ -278,7 +326,9 @@ private[parquet] object FileSystemHelper { fs.listStatus(path).map(_.getPath) } - // finds the maximum taskid in the output file names at the given path + /** + * Finds the maximum taskid in the output file names at the given path. + */ def findMaxTaskId(pathStr: String, conf: Configuration): Int = { val files = FileSystemHelper.listFiles(pathStr, conf) // filename pattern is part-r-.parquet diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index f37976f7313c1..46c7172985642 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -19,15 +19,34 @@ package org.apache.spark.sql.parquet import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce.Job +import parquet.example.data.{GroupWriter, Group} +import parquet.example.data.simple.SimpleGroup import parquet.hadoop.ParquetWriter -import parquet.hadoop.util.ContextUtil +import parquet.hadoop.api.WriteSupport +import parquet.hadoop.api.WriteSupport.WriteContext +import parquet.io.api.RecordConsumer import parquet.schema.{MessageType, MessageTypeParser} -import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.util.Utils +// Write support class for nested groups: ParquetWriter initializes GroupWriteSupport +// with an empty configuration (it is after all not intended to be used in this way?) +// and members are private so we need to make our own in order to pass the schema +// to the writer. +private class TestGroupWriteSupport(schema: MessageType) extends WriteSupport[Group] { + var groupWriter: GroupWriter = null + override def prepareForWrite(recordConsumer: RecordConsumer): Unit = { + groupWriter = new GroupWriter(recordConsumer, schema) + } + override def init(configuration: Configuration): WriteContext = { + new WriteContext(schema, new java.util.HashMap[String, String]()) + } + override def write(record: Group) { + groupWriter.write(record) + } +} + private[sql] object ParquetTestData { val testSchema = @@ -43,7 +62,7 @@ private[sql] object ParquetTestData { // field names for test assertion error messages val testSchemaFieldNames = Seq( "myboolean:Boolean", - "mtint:Int", + "myint:Int", "mystring:String", "mylong:Long", "myfloat:Float", @@ -58,6 +77,18 @@ private[sql] object ParquetTestData { |} """.stripMargin + val testFilterSchema = + """ + |message myrecord { + |required boolean myboolean; + |required int32 myint; + |required binary mystring; + |required int64 mylong; + |required float myfloat; + |required double mydouble; + |} + """.stripMargin + // field names for test assertion error messages val subTestSchemaFieldNames = Seq( "myboolean:Boolean", @@ -65,36 +96,57 @@ private[sql] object ParquetTestData { ) val testDir = Utils.createTempDir() + val testFilterDir = Utils.createTempDir() lazy val testData = new ParquetRelation(testDir.toURI.toString) def writeFile() = { testDir.delete val path: Path = new Path(new Path(testDir.toURI), new Path("part-r-0.parquet")) - val job = new Job() - val configuration: Configuration = ContextUtil.getConfiguration(job) val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) + val writeSupport = new TestGroupWriteSupport(schema) + val writer = new ParquetWriter[Group](path, writeSupport) - val writeSupport = new RowWriteSupport() - writeSupport.setSchema(schema, configuration) - val writer = new ParquetWriter(path, writeSupport) for(i <- 0 until 15) { - val data = new Array[Any](6) + val record = new SimpleGroup(schema) if (i % 3 == 0) { - data.update(0, true) + record.add(0, true) } else { - data.update(0, false) + record.add(0, false) } if (i % 5 == 0) { - data.update(1, 5) + record.add(1, 5) + } + record.add(2, "abc") + record.add(3, i.toLong << 33) + record.add(4, 2.5F) + record.add(5, 4.5D) + writer.write(record) + } + writer.close() + } + + def writeFilterFile(records: Int = 200) = { + // for microbenchmark use: records = 300000000 + testFilterDir.delete + val path: Path = new Path(new Path(testFilterDir.toURI), new Path("part-r-0.parquet")) + val schema: MessageType = MessageTypeParser.parseMessageType(testFilterSchema) + val writeSupport = new TestGroupWriteSupport(schema) + val writer = new ParquetWriter[Group](path, writeSupport) + + for(i <- 0 to records) { + val record = new SimpleGroup(schema) + if (i % 4 == 0) { + record.add(0, true) } else { - data.update(1, null) // optional + record.add(0, false) } - data.update(2, "abc") - data.update(3, i.toLong << 33) - data.update(4, 2.5F) - data.update(5, 4.5D) - writer.write(new GenericRow(data.toArray)) + record.add(1, i) + record.add(2, i.toString) + record.add(3, i.toLong) + record.add(4, i.toFloat + 0.5f) + record.add(5, i.toDouble + 0.5d) + writer.write(record) } writer.close() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index ff1677eb8a480..65f4c17aeee3a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -17,25 +17,25 @@ package org.apache.spark.sql.parquet -import java.io.File - import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.hadoop.mapreduce.Job import parquet.hadoop.ParquetFileWriter -import parquet.schema.MessageTypeParser import parquet.hadoop.util.ContextUtil +import parquet.schema.MessageTypeParser import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.util.getTempFilePath -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.TestData +import org.apache.spark.sql.SchemaRDD +import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.catalyst.expressions.Equals +import org.apache.spark.sql.catalyst.types.IntegerType import org.apache.spark.util.Utils -import org.apache.spark.sql.catalyst.types.{StringType, IntegerType, DataType} -import org.apache.spark.sql.{parquet, SchemaRDD} // Implicits import org.apache.spark.sql.test.TestSQLContext._ @@ -64,12 +64,16 @@ class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { override def beforeAll() { ParquetTestData.writeFile() + ParquetTestData.writeFilterFile() testRDD = parquetFile(ParquetTestData.testDir.toString) testRDD.registerAsTable("testsource") + parquetFile(ParquetTestData.testFilterDir.toString) + .registerAsTable("testfiltersource") } override def afterAll() { Utils.deleteRecursively(ParquetTestData.testDir) + Utils.deleteRecursively(ParquetTestData.testFilterDir) // here we should also unregister the table?? } @@ -120,7 +124,7 @@ class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { val scanner = new ParquetTableScan( ParquetTestData.testData.output, ParquetTestData.testData, - None)(TestSQLContext.sparkContext) + Seq())(TestSQLContext.sparkContext) val projected = scanner.pruneColumns(ParquetTypesConverter .convertToAttributes(MessageTypeParser .parseMessageType(ParquetTestData.subTestSchema))) @@ -196,7 +200,6 @@ class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { assert(true) } - test("insert (appending) to same table via Scala API") { sql("INSERT INTO testsource SELECT * FROM testsource").collect() val double_rdd = sql("SELECT * FROM testsource").collect() @@ -239,5 +242,121 @@ class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { Utils.deleteRecursively(file) assert(true) } + + test("create RecordFilter for simple predicates") { + val attribute1 = new AttributeReference("first", IntegerType, false)() + val predicate1 = new Equals(attribute1, new Literal(1, IntegerType)) + val filter1 = ParquetFilters.createFilter(predicate1) + assert(filter1.isDefined) + assert(filter1.get.predicate == predicate1, "predicates do not match") + assert(filter1.get.isInstanceOf[ComparisonFilter]) + val cmpFilter1 = filter1.get.asInstanceOf[ComparisonFilter] + assert(cmpFilter1.columnName == "first", "column name incorrect") + + val predicate2 = new LessThan(attribute1, new Literal(4, IntegerType)) + val filter2 = ParquetFilters.createFilter(predicate2) + assert(filter2.isDefined) + assert(filter2.get.predicate == predicate2, "predicates do not match") + assert(filter2.get.isInstanceOf[ComparisonFilter]) + val cmpFilter2 = filter2.get.asInstanceOf[ComparisonFilter] + assert(cmpFilter2.columnName == "first", "column name incorrect") + + val predicate3 = new And(predicate1, predicate2) + val filter3 = ParquetFilters.createFilter(predicate3) + assert(filter3.isDefined) + assert(filter3.get.predicate == predicate3, "predicates do not match") + assert(filter3.get.isInstanceOf[AndFilter]) + + val predicate4 = new Or(predicate1, predicate2) + val filter4 = ParquetFilters.createFilter(predicate4) + assert(filter4.isDefined) + assert(filter4.get.predicate == predicate4, "predicates do not match") + assert(filter4.get.isInstanceOf[OrFilter]) + + val attribute2 = new AttributeReference("second", IntegerType, false)() + val predicate5 = new GreaterThan(attribute1, attribute2) + val badfilter = ParquetFilters.createFilter(predicate5) + assert(badfilter.isDefined === false) + } + + test("test filter by predicate pushdown") { + for(myval <- Seq("myint", "mylong", "mydouble", "myfloat")) { + println(s"testing field $myval") + val query1 = sql(s"SELECT * FROM testfiltersource WHERE $myval < 150 AND $myval >= 100") + assert( + query1.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val result1 = query1.collect() + assert(result1.size === 50) + assert(result1(0)(1) === 100) + assert(result1(49)(1) === 149) + val query2 = sql(s"SELECT * FROM testfiltersource WHERE $myval > 150 AND $myval <= 200") + assert( + query2.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val result2 = query2.collect() + assert(result2.size === 50) + if (myval == "myint" || myval == "mylong") { + assert(result2(0)(1) === 151) + assert(result2(49)(1) === 200) + } else { + assert(result2(0)(1) === 150) + assert(result2(49)(1) === 199) + } + } + for(myval <- Seq("myint", "mylong")) { + val query3 = sql(s"SELECT * FROM testfiltersource WHERE $myval > 190 OR $myval < 10") + assert( + query3.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val result3 = query3.collect() + assert(result3.size === 20) + assert(result3(0)(1) === 0) + assert(result3(9)(1) === 9) + assert(result3(10)(1) === 191) + assert(result3(19)(1) === 200) + } + for(myval <- Seq("mydouble", "myfloat")) { + val result4 = + if (myval == "mydouble") { + val query4 = sql(s"SELECT * FROM testfiltersource WHERE $myval > 190.5 OR $myval < 10.0") + assert( + query4.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + query4.collect() + } else { + // CASTs are problematic. Here myfloat will be casted to a double and it seems there is + // currently no way to specify float constants in SqlParser? + sql(s"SELECT * FROM testfiltersource WHERE $myval > 190.5 OR $myval < 10").collect() + } + assert(result4.size === 20) + assert(result4(0)(1) === 0) + assert(result4(9)(1) === 9) + assert(result4(10)(1) === 191) + assert(result4(19)(1) === 200) + } + val query5 = sql(s"SELECT * FROM testfiltersource WHERE myboolean = true AND myint < 40") + assert( + query5.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val booleanResult = query5.collect() + assert(booleanResult.size === 10) + for(i <- 0 until 10) { + if (!booleanResult(i).getBoolean(0)) { + fail(s"Boolean value in result row $i not true") + } + if (booleanResult(i).getInt(1) != i * 4) { + fail(s"Int value in result row $i should be ${4*i}") + } + } + val query6 = sql("SELECT * FROM testfiltersource WHERE mystring = \"100\"") + assert( + query6.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val stringResult = query6.collect() + assert(stringResult.size === 1) + assert(stringResult(0).getString(2) == "100", "stringvalue incorrect") + assert(stringResult(0).getInt(1) === 100) + } } From fed6303f29250bd5e656dbdd731b38938c933a61 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 16 May 2014 17:35:05 -0700 Subject: [PATCH 383/641] Tweaks to Mesos docs - Mention Apache downloads first - Shorten some wording Author: Matei Zaharia Closes #806 from mateiz/doc-update and squashes the following commits: d9345cd [Matei Zaharia] typo a179f8d [Matei Zaharia] Tweaks to Mesos docs --- docs/running-on-mesos.md | 71 +++++++++++++++++++--------------------- 1 file changed, 34 insertions(+), 37 deletions(-) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index ef762aa7b8fcc..df8687f81f3ef 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -3,16 +3,15 @@ layout: global title: Running Spark on Mesos --- -# Why Mesos - Spark can run on hardware clusters managed by [Apache Mesos](http://mesos.apache.org/). The advantages of deploying Spark with Mesos include: + - dynamic partitioning between Spark and other [frameworks](https://mesos.apache.org/documentation/latest/mesos-frameworks/) - scalable partitioning between multiple instances of Spark -# How it works +# How it Works In a standalone cluster deployment, the cluster manager in the below diagram is a Spark master instance. When using Mesos, the Mesos master replaces the Spark master as the cluster manager. @@ -37,11 +36,25 @@ require any special patches of Mesos. If you already have a Mesos cluster running, you can skip this Mesos installation step. Otherwise, installing Mesos for Spark is no different than installing Mesos for use by other -frameworks. You can install Mesos using either prebuilt packages or by compiling from source. +frameworks. You can install Mesos either from source or using prebuilt packages. + +## From Source + +To install Apache Mesos from source, follow these steps: + +1. Download a Mesos release from a + [mirror](http://www.apache.org/dyn/closer.cgi/mesos/{{site.MESOS_VERSION}}/) +2. Follow the Mesos [Getting Started](http://mesos.apache.org/gettingstarted) page for compiling and + installing Mesos + +**Note:** If you want to run Mesos without installing it into the default paths on your system +(e.g., if you lack administrative privileges to install it), pass the +`--prefix` option to `configure` to tell it where to install. For example, pass +`--prefix=/home/me/mesos`. By default the prefix is `/usr/local`. -## Prebuilt packages +## Third-Party Packages -The Apache Mesos project only publishes source package releases, no binary releases. But other +The Apache Mesos project only publishes source releases, not binary packages. But other third party projects publish binary releases that may be helpful in setting Mesos up. One of those is Mesosphere. To install Mesos using the binary releases provided by Mesosphere: @@ -52,20 +65,6 @@ One of those is Mesosphere. To install Mesos using the binary releases provided The Mesosphere installation documents suggest setting up ZooKeeper to handle Mesos master failover, but Mesos can be run without ZooKeeper using a single master as well. -## From source - -To install Mesos directly from the upstream project rather than a third party, install from source. - -1. Download the Mesos distribution from a - [mirror](http://www.apache.org/dyn/closer.cgi/mesos/{{site.MESOS_VERSION}}/) -2. Follow the Mesos [Getting Started](http://mesos.apache.org/gettingstarted) page for compiling and - installing Mesos - -**Note:** If you want to run Mesos without installing it into the default paths on your system -(e.g., if you lack administrative privileges to install it), you should also pass the -`--prefix` option to `configure` to tell it where to install. For example, pass -`--prefix=/home/user/mesos`. By default the prefix is `/usr/local`. - ## Verification To verify that the Mesos cluster is ready for Spark, navigate to the Mesos master webui at port @@ -74,32 +73,30 @@ To verify that the Mesos cluster is ready for Spark, navigate to the Mesos maste # Connecting Spark to Mesos -To use Mesos from Spark, you need a Spark distribution available in a place accessible by Mesos, and +To use Mesos from Spark, you need a Spark binary package available in a place accessible by Mesos, and a Spark driver program configured to connect to Mesos. -## Uploading Spark Distribution - -When Mesos runs a task on a Mesos slave for the first time, that slave must have a distribution of -Spark available for running the Spark Mesos executor backend. A distribution of Spark is just a -compiled binary version of Spark. +## Uploading Spark Package -The Spark distribution can be hosted at any Hadoop URI, including HTTP via `http://`, [Amazon Simple -Storage Service](http://aws.amazon.com/s3) via `s3://`, or HDFS via `hdfs:///`. +When Mesos runs a task on a Mesos slave for the first time, that slave must have a Spark binary +package for running the Spark Mesos executor backend. +The Spark package can be hosted at any Hadoop-accessible URI, including HTTP via `http://`, +[Amazon Simple Storage Service](http://aws.amazon.com/s3) via `s3n://`, or HDFS via `hdfs://`. -To use a precompiled distribution: +To use a precompiled package: -1. Download a Spark distribution from the Spark [download page](https://spark.apache.org/downloads.html) +1. Download a Spark binary package from the Spark [download page](https://spark.apache.org/downloads.html) 2. Upload to hdfs/http/s3 To host on HDFS, use the Hadoop fs put command: `hadoop fs -put spark-{{site.SPARK_VERSION}}.tar.gz /path/to/spark-{{site.SPARK_VERSION}}.tar.gz` -Or if you are using a custom-compiled version of Spark, you will need to create a distribution using +Or if you are using a custom-compiled version of Spark, you will need to create a package using the `make-distribution.sh` script included in a Spark source tarball/checkout. 1. Download and build Spark using the instructions [here](index.html) -2. Create a Spark distribution using `make-distribution.sh --tgz`. +2. Create a binary package using `make-distribution.sh --tgz`. 3. Upload archive to http/s3/hdfs @@ -115,8 +112,8 @@ The driver also needs some configuration in `spark-env.sh` to interact properly `/lib/libmesos.so` where the prefix is `/usr/local` by default. See Mesos installation instructions above. On Mac OS X, the library is called `libmesos.dylib` instead of `libmesos.so`. - * `export SPARK_EXECUTOR_URI=`. -2. Also set `spark.executor.uri` to + * `export SPARK_EXECUTOR_URI=`. +2. Also set `spark.executor.uri` to ``. Now when starting a Spark application against the cluster, pass a `mesos://` or `zk://` URL as the master when creating a `SparkContext`. For example: @@ -129,7 +126,7 @@ val conf = new SparkConf() val sc = new SparkContext(conf) {% endhighlight %} -When running a shell the `spark.executor.uri` parameter is inherited from `SPARK_EXECUTOR_URI`, so +When running a shell, the `spark.executor.uri` parameter is inherited from `SPARK_EXECUTOR_URI`, so it does not need to be redundantly passed in as a system property. {% highlight bash %} @@ -168,7 +165,7 @@ using `conf.set("spark.cores.max", "10")` (for example). # Running Alongside Hadoop You can run Spark and Mesos alongside your existing Hadoop cluster by just launching them as a -separate service on the machines. To access Hadoop data from Spark, a full hdfs:// URL is required +separate service on the machines. To access Hadoop data from Spark, a full `hdfs://` URL is required (typically `hdfs://:9000/path`, but you can find the right URL on your Hadoop Namenode web UI). @@ -195,7 +192,7 @@ A few places to look during debugging: And common pitfalls: - Spark assembly not reachable/accessible - - Slaves need to be able to download the distribution + - Slaves must be able to download the Spark binary package from the `http://`, `hdfs://` or `s3n://` URL you gave - Firewall blocking communications - Check for messages about failed connections - Temporarily disable firewalls for debugging and then poke appropriate holes From a80a6a139e729ee3f81ec4f0028e084d2d9f7e82 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 16 May 2014 20:25:10 -0700 Subject: [PATCH 384/641] SPARK-1864 Look in spark conf instead of system properties when propagating configuration to executors. Author: Michael Armbrust Closes #808 from marmbrus/confClasspath and squashes the following commits: 4c31d57 [Michael Armbrust] Look in spark conf instead of system properties when propagating configuration to executors. --- .../scheduler/cluster/SparkDeploySchedulerBackend.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 933f6e0571518..9768670855f80 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -46,12 +46,13 @@ private[spark] class SparkDeploySchedulerBackend( CoarseGrainedSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") - val classPathEntries = sys.props.get("spark.executor.extraClassPath").toSeq.flatMap { cp => - cp.split(java.io.File.pathSeparator) - } - val libraryPathEntries = sys.props.get("spark.executor.extraLibraryPath").toSeq.flatMap { cp => + val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath").toSeq.flatMap { cp => cp.split(java.io.File.pathSeparator) } + val libraryPathEntries = + sc.conf.getOption("spark.executor.extraLibraryPath").toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, From c0ab85d7320cea90e6331fb03a70349bc804c1b1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 16 May 2014 21:42:14 -0700 Subject: [PATCH 385/641] Version bump of spark-ec2 scripts This will allow us to change things in spark-ec2 related to the 1.0 release. Author: Patrick Wendell Closes #809 from pwendell/spark-ec2 and squashes the following commits: 59117fb [Patrick Wendell] Version bump of spark-ec2 scripts --- ec2/spark_ec2.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 0a8d6ca5d7519..4c5a99f5f6b2e 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -443,7 +443,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): # NOTE: We should clone the repository before running deploy_files to # prevent ec2-variables.sh from being overwritten - ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/mesos/spark-ec2.git -b v2") + ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/mesos/spark-ec2.git -b v3") print "Deploying files to master..." deploy_files(conn, "deploy.generic", opts, master_nodes, slave_nodes, modules) From 4b8ec6fcfd7a7ef0857d5b21917183c181301c95 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 16 May 2014 22:34:38 -0700 Subject: [PATCH 386/641] [SPARK-1808] Route bin/pyspark through Spark submit **Problem.** For `bin/pyspark`, there is currently no other way to specify Spark configuration properties other than through `SPARK_JAVA_OPTS` in `conf/spark-env.sh`. However, this mechanism is supposedly deprecated. Instead, it needs to pick up configurations explicitly specified in `conf/spark-defaults.conf`. **Solution.** Have `bin/pyspark` invoke `bin/spark-submit`, like all of its counterparts in Scala land (i.e. `bin/spark-shell`, `bin/run-example`). This has the additional benefit of making the invocation of all the user facing Spark scripts consistent. **Details.** `bin/pyspark` inherently handles two cases: (1) running python applications and (2) running the python shell. For (1), Spark submit already handles running python applications. For cases in which `bin/pyspark` is given a python file, we can simply call pass the file directly to Spark submit and let it handle the rest. For case (2), `bin/pyspark` starts a python process as before, which launches the JVM as a sub-process. The existing code already provides a code path to do this. All we needed to change is to use `bin/spark-submit` instead of `spark-class` to launch the JVM. This requires modifications to Spark submit to handle the pyspark shell as a special case. This has been tested locally (OSX and Windows 7), on a standalone cluster, and on a YARN cluster. Running IPython also works as before, except now it takes in Spark submit arguments too. Author: Andrew Or Closes #799 from andrewor14/pyspark-submit and squashes the following commits: bf37e36 [Andrew Or] Minor changes 01066fa [Andrew Or] bin/pyspark for Windows c8cb3bf [Andrew Or] Handle perverse app names (with escaped quotes) 1866f85 [Andrew Or] Windows is not cooperating 456d844 [Andrew Or] Guard against shlex hanging if PYSPARK_SUBMIT_ARGS is not set 7eebda8 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-submit b7ba0d8 [Andrew Or] Address a few comments (minor) 06eb138 [Andrew Or] Use shlex instead of writing our own parser 05879fa [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-submit a823661 [Andrew Or] Fix --die-on-broken-pipe not propagated properly 6fba412 [Andrew Or] Deal with quotes + address various comments fe4c8a7 [Andrew Or] Update --help for bin/pyspark afe47bf [Andrew Or] Fix spark shell f04aaa4 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-submit a371d26 [Andrew Or] Route bin/pyspark through Spark submit --- bin/pyspark | 35 ++++++++++-- bin/pyspark2.cmd | 21 ++++++- bin/spark-shell | 6 +- bin/spark-shell.cmd | 2 +- .../apache/spark/deploy/PythonRunner.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 55 ++++++++++++++----- .../spark/deploy/SparkSubmitArguments.scala | 6 +- .../scala/org/apache/spark/util/Utils.scala | 2 +- python/pyspark/java_gateway.py | 10 ++-- python/pyspark/shell.py | 2 +- 10 files changed, 107 insertions(+), 34 deletions(-) diff --git a/bin/pyspark b/bin/pyspark index 10e35e0f1734e..9e1364e44c8c4 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -25,6 +25,12 @@ export SPARK_HOME="$FWDIR" SCALA_VERSION=2.10 +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + echo "Usage: ./bin/pyspark [options]" + ./bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 +fi + # Exit if the user hasn't compiled Spark if [ ! -f "$FWDIR/RELEASE" ]; then # Exit if the user hasn't compiled Spark @@ -52,13 +58,34 @@ export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.1-src.zip:$PYTHONPATH export OLD_PYTHONSTARTUP=$PYTHONSTARTUP export PYTHONSTARTUP=$FWDIR/python/pyspark/shell.py +# If IPython options are specified, assume user wants to run IPython if [ -n "$IPYTHON_OPTS" ]; then IPYTHON=1 fi -# Only use ipython if no command line arguments were provided [SPARK-1134] -if [[ "$IPYTHON" = "1" && $# = 0 ]] ; then - exec ipython $IPYTHON_OPTS +# Build up arguments list manually to preserve quotes and backslashes. +# We export Spark submit arguments as an environment variable because shell.py must run as a +# PYTHONSTARTUP script, which does not take in arguments. This is required for IPython notebooks. + +PYSPARK_SUBMIT_ARGS="" +whitespace="[[:space:]]" +for i in "$@"; do + if [[ $i =~ \" ]]; then i=$(echo $i | sed 's/\"/\\\"/g'); fi + if [[ $i =~ $whitespace ]]; then i=\"$i\"; fi + PYSPARK_SUBMIT_ARGS="$PYSPARK_SUBMIT_ARGS $i" +done +export PYSPARK_SUBMIT_ARGS + +# If a python file is provided, directly run spark-submit. +if [[ "$1" =~ \.py$ ]]; then + echo -e "\nWARNING: Running python applications through ./bin/pyspark is deprecated as of Spark 1.0." 1>&2 + echo -e "Use ./bin/spark-submit \n" 1>&2 + exec $FWDIR/bin/spark-submit "$@" else - exec "$PYSPARK_PYTHON" "$@" + # Only use ipython if no command line arguments were provided [SPARK-1134] + if [[ "$IPYTHON" = "1" ]]; then + exec ipython $IPYTHON_OPTS + else + exec "$PYSPARK_PYTHON" + fi fi diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index d7cfd5eec501c..0ef9eea95342e 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -31,7 +31,7 @@ set FOUND_JAR=0 for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( set FOUND_JAR=1 ) -if "%FOUND_JAR%"=="0" ( +if [%FOUND_JAR%] == [0] ( echo Failed to find Spark assembly JAR. echo You need to build Spark with sbt\sbt assembly before running this program. goto exit @@ -42,15 +42,30 @@ rem Load environment variables from conf\spark-env.cmd, if it exists if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Figure out which Python to use. -if "x%PYSPARK_PYTHON%"=="x" set PYSPARK_PYTHON=python +if [%PYSPARK_PYTHON%] == [] set PYSPARK_PYTHON=python set PYTHONPATH=%FWDIR%python;%PYTHONPATH% set PYTHONPATH=%FWDIR%python\lib\py4j-0.8.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%FWDIR%python\pyspark\shell.py +set PYSPARK_SUBMIT_ARGS=%* echo Running %PYSPARK_PYTHON% with PYTHONPATH=%PYTHONPATH% -"%PYSPARK_PYTHON%" %* +rem Check whether the argument is a file +for /f %%i in ('echo %1^| findstr /R "\.py"') do ( + set PYTHON_FILE=%%i +) + +if [%PYTHON_FILE%] == [] ( + %PYSPARK_PYTHON% +) else ( + echo. + echo WARNING: Running python applications through ./bin/pyspark.cmd is deprecated as of Spark 1.0. + echo Use ./bin/spark-submit ^ + echo. + "%FWDIR%\bin\spark-submit.cmd" %PYSPARK_SUBMIT_ARGS% +) + :exit diff --git a/bin/spark-shell b/bin/spark-shell index 7f03349c5e910..c158683ab3f99 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -28,7 +28,7 @@ esac # Enter posix mode for bash set -o posix -if [[ "$@" == *--help* ]]; then +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then echo "Usage: ./bin/spark-shell [options]" ./bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 exit 0 @@ -46,11 +46,11 @@ function main(){ # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-submit spark-internal "$@" --class org.apache.spark.repl.Main + $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main stty icanon echo > /dev/null 2>&1 else export SPARK_SUBMIT_OPTS - $FWDIR/bin/spark-submit spark-internal "$@" --class org.apache.spark.repl.Main + $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main fi } diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd index ca0c722c926f5..4b9708a8c03f3 100755 --- a/bin/spark-shell.cmd +++ b/bin/spark-shell.cmd @@ -19,4 +19,4 @@ rem set SPARK_HOME=%~dp0.. -cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-internal %* --class org.apache.spark.repl.Main +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell %* --class org.apache.spark.repl.Main diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala index e20d4486c8f0c..2dfa02bd26f13 100644 --- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -42,7 +42,7 @@ object PythonRunner { // Build up a PYTHONPATH that includes the Spark assembly JAR (where this class is), the // python directories in SPARK_HOME (if set), and any files in the pyFiles argument val pathElements = new ArrayBuffer[String] - pathElements ++= pyFiles.split(",") + pathElements ++= Option(pyFiles).getOrElse("").split(",") pathElements += PythonUtils.sparkPythonPath pathElements += sys.env.getOrElse("PYTHONPATH", "") val pythonPath = PythonUtils.mergePythonPaths(pathElements: _*) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index e86182e4c56ce..a99b2176e2b5e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -41,10 +41,10 @@ object SparkSubmit { private var clusterManager: Int = LOCAL /** - * A special jar name that indicates the class being run is inside of Spark itself, - * and therefore no user jar is needed. + * Special primary resource names that represent shells rather than application jars. */ - private val RESERVED_JAR_NAME = "spark-internal" + private val SPARK_SHELL = "spark-shell" + private val PYSPARK_SHELL = "pyspark-shell" def main(args: Array[String]) { val appArgs = new SparkSubmitArguments(args) @@ -71,8 +71,8 @@ object SparkSubmit { * entries for the child, a list of system properties, a list of env vars * and the main class for the child */ - private[spark] def createLaunchEnv(args: SparkSubmitArguments): (ArrayBuffer[String], - ArrayBuffer[String], Map[String, String], String) = { + private[spark] def createLaunchEnv(args: SparkSubmitArguments) + : (ArrayBuffer[String], ArrayBuffer[String], Map[String, String], String) = { if (args.master.startsWith("local")) { clusterManager = LOCAL } else if (args.master.startsWith("yarn")) { @@ -121,24 +121,30 @@ object SparkSubmit { printErrorAndExit("Cannot currently run driver on the cluster in Mesos") } - // If we're running a Python app, set the Java class to run to be our PythonRunner, add - // Python files to deployment list, and pass the main file and Python path to PythonRunner + // If we're running a python app, set the main class to our specific python runner if (isPython) { if (deployOnCluster) { printErrorAndExit("Cannot currently run Python driver programs on cluster") } - args.mainClass = "org.apache.spark.deploy.PythonRunner" - args.files = mergeFileLists(args.files, args.pyFiles, args.primaryResource) + if (args.primaryResource == PYSPARK_SHELL) { + args.mainClass = "py4j.GatewayServer" + args.childArgs = ArrayBuffer("--die-on-broken-pipe", "0") + } else { + // If a python file is provided, add it to the child arguments and list of files to deploy. + // Usage: PythonAppRunner
      [app arguments] + args.mainClass = "org.apache.spark.deploy.PythonRunner" + args.childArgs = ArrayBuffer(args.primaryResource, args.pyFiles) ++ args.childArgs + args.files = mergeFileLists(args.files, args.primaryResource) + } val pyFiles = Option(args.pyFiles).getOrElse("") - args.childArgs = ArrayBuffer(args.primaryResource, pyFiles) ++ args.childArgs - args.primaryResource = RESERVED_JAR_NAME + args.files = mergeFileLists(args.files, pyFiles) sysProps("spark.submit.pyFiles") = pyFiles } // If we're deploying into YARN, use yarn.Client as a wrapper around the user class if (!deployOnCluster) { childMainClass = args.mainClass - if (args.primaryResource != RESERVED_JAR_NAME) { + if (isUserJar(args.primaryResource)) { childClasspath += args.primaryResource } } else if (clusterManager == YARN) { @@ -219,7 +225,7 @@ object SparkSubmit { // For python files, the primary resource is already distributed as a regular file if (!isYarnCluster && !isPython) { var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq()) - if (args.primaryResource != RESERVED_JAR_NAME) { + if (isUserJar(args.primaryResource)) { jars = jars ++ Seq(args.primaryResource) } sysProps.put("spark.jars", jars.mkString(",")) @@ -293,7 +299,7 @@ object SparkSubmit { } private def addJarToClasspath(localJar: String, loader: ExecutorURLClassLoader) { - val localJarFile = new File(new URI(localJar).getPath()) + val localJarFile = new File(new URI(localJar).getPath) if (!localJarFile.exists()) { printWarning(s"Jar $localJar does not exist, skipping.") } @@ -302,6 +308,27 @@ object SparkSubmit { loader.addURL(url) } + /** + * Return whether the given primary resource represents a user jar. + */ + private def isUserJar(primaryResource: String): Boolean = { + !isShell(primaryResource) && !isPython(primaryResource) + } + + /** + * Return whether the given primary resource represents a shell. + */ + private def isShell(primaryResource: String): Boolean = { + primaryResource == SPARK_SHELL || primaryResource == PYSPARK_SHELL + } + + /** + * Return whether the given primary resource requires running python. + */ + private[spark] def isPython(primaryResource: String): Boolean = { + primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL + } + /** * Merge a sequence of comma-separated file lists, some of which may be null to indicate * no files, into a single comma-separated string. diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 2d327aa3fb27f..264d4544cd31c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -298,11 +298,13 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { case v => primaryResource = v inSparkOpts = false - isPython = v.endsWith(".py") + isPython = SparkSubmit.isPython(v) parse(tail) } } else { - childArgs += value + if (!value.isEmpty) { + childArgs += value + } parse(tail) } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 388f7222428db..0c7cff019fce1 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1101,7 +1101,7 @@ private[spark] object Utils extends Logging { * Strip the directory from a path name */ def stripDirectory(path: String): String = { - path.split(File.separator).last + new File(path).getName } /** diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 3d0936fdca911..91ae8263f66b8 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -18,12 +18,12 @@ import os import sys import signal +import shlex import platform from subprocess import Popen, PIPE from threading import Thread from py4j.java_gateway import java_import, JavaGateway, GatewayClient - def launch_gateway(): SPARK_HOME = os.environ["SPARK_HOME"] @@ -34,9 +34,11 @@ def launch_gateway(): # Launch the Py4j gateway using Spark's run command so that we pick up the # proper classpath and settings from spark-env.sh on_windows = platform.system() == "Windows" - script = "./bin/spark-class.cmd" if on_windows else "./bin/spark-class" - command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer", - "--die-on-broken-pipe", "0"] + script = "./bin/spark-submit.cmd" if on_windows else "./bin/spark-submit" + submit_args = os.environ.get("PYSPARK_SUBMIT_ARGS") + submit_args = submit_args if submit_args is not None else "" + submit_args = shlex.split(submit_args) + command = [os.path.join(SPARK_HOME, script), "pyspark-shell"] + submit_args if not on_windows: # Don't send ctrl-c / SIGINT to the Java gateway: def preexec_func(): diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index d172d588bfbd8..ebd714db7a918 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -40,7 +40,7 @@ if os.environ.get("SPARK_EXECUTOR_URI"): SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"]) -sc = SparkContext(os.environ.get("MASTER", "local[*]"), "PySparkShell", pyFiles=add_files) +sc = SparkContext(appName="PySparkShell", pyFiles=add_files) print("""Welcome to ____ __ From cf6cbe9f76c3b322a968c836d039fc5b70d4ce43 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 16 May 2014 22:36:23 -0700 Subject: [PATCH 387/641] [SPARK-1824] Remove from Python examples A recent PR (#552) fixed this for all Scala / Java examples. We need to do it for python too. Note that this blocks on #799, which makes `bin/pyspark` go through Spark submit. With only the changes in this PR, the only way to run these examples is through Spark submit. Once #799 goes in, you can use `bin/pyspark` to run them too. For example, ``` bin/pyspark examples/src/main/python/pi.py 100 --master local-cluster[4,1,512] ``` Author: Andrew Or Closes #802 from andrewor14/python-examples and squashes the following commits: cf50b9f [Andrew Or] De-indent python comments (minor) 50f80b1 [Andrew Or] Remove pyFiles from SparkContext construction c362f69 [Andrew Or] Update docs to use spark-submit for python applications 7072c6a [Andrew Or] Merge branch 'master' of github.com:apache/spark into python-examples 427a5f0 [Andrew Or] Update docs d32072c [Andrew Or] Remove from examples + update usages --- docs/index.md | 11 ++++--- docs/python-programming-guide.md | 32 ++++++++++--------- examples/src/main/python/als.py | 18 +++++------ examples/src/main/python/kmeans.py | 12 +++---- .../src/main/python/logistic_regression.py | 10 +++--- examples/src/main/python/mllib/kmeans.py | 10 +++--- .../main/python/mllib/logistic_regression.py | 10 +++--- examples/src/main/python/pagerank.py | 10 +++--- examples/src/main/python/pi.py | 10 +++--- examples/src/main/python/sort.py | 8 ++--- .../src/main/python/transitive_closure.py | 10 +++--- examples/src/main/python/wordcount.py | 8 ++--- 12 files changed, 77 insertions(+), 72 deletions(-) diff --git a/docs/index.md b/docs/index.md index 48182a27d28ae..c9b10376cc809 100644 --- a/docs/index.md +++ b/docs/index.md @@ -43,12 +43,15 @@ The `--master` option specifies the locally with one thread, or `local[N]` to run locally with N threads. You should start by using `local` for testing. For a full list of options, run Spark shell with the `--help` option. -Spark also provides a Python interface. To run an example Spark application written in Python, use -`bin/pyspark [params]`. For example, +Spark also provides a Python interface. To run Spark interactively in a Python interpreter, use +`bin/pyspark`. As in Spark shell, you can also pass in the `--master` option to configure your +master URL. - ./bin/pyspark examples/src/main/python/pi.py local[2] 10 + ./bin/pyspark --master local[2] -or simply `bin/pyspark` without any arguments to run Spark interactively in a python interpreter. +Example applications are also provided in Python. For example, + + ./bin/spark-submit examples/src/main/python/pi.py 10 # Launching on a Cluster diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 17675acba6bb8..b686bee1aebe2 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -60,13 +60,9 @@ By default, PySpark requires `python` to be available on the system `PATH` and u All of PySpark's library dependencies, including [Py4J](http://py4j.sourceforge.net/), are bundled with PySpark and automatically imported. -Standalone PySpark applications should be run using the `bin/spark-submit` script, which automatically -configures the Java and Python environment for running Spark. - - # Interactive Use -The `bin/pyspark` script launches a Python interpreter that is configured to run PySpark applications. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options: +The `bin/pyspark` script launches a Python interpreter that is configured to run PySpark applications. To use `pyspark` interactively, first build Spark, then launch it directly from the command line: {% highlight bash %} $ sbt/sbt assembly @@ -83,20 +79,24 @@ The Python shell can be used explore data interactively and is a simple way to l {% endhighlight %} By default, the `bin/pyspark` shell creates SparkContext that runs applications locally on all of -your machine's logical cores. -To connect to a non-local cluster, or to specify a number of cores, set the `MASTER` environment variable. -For example, to use the `bin/pyspark` shell with a [standalone Spark cluster](spark-standalone.html): +your machine's logical cores. To connect to a non-local cluster, or to specify a number of cores, +set the `--master` flag. For example, to use the `bin/pyspark` shell with a +[standalone Spark cluster](spark-standalone.html): {% highlight bash %} -$ MASTER=spark://IP:PORT ./bin/pyspark +$ ./bin/pyspark --master spark://1.2.3.4:7077 {% endhighlight %} Or, to use exactly four cores on the local machine: {% highlight bash %} -$ MASTER=local[4] ./bin/pyspark +$ ./bin/pyspark --master local[4] {% endhighlight %} +Under the hood `bin/pyspark` is a wrapper around the +[Spark submit script](cluster-overview.html#launching-applications-with-spark-submit), so these +two scripts share the same list of options. For a complete list of options, run `bin/pyspark` with +the `--help` option. ## IPython @@ -115,13 +115,14 @@ the [IPython Notebook](http://ipython.org/notebook.html) with PyLab graphing sup $ IPYTHON_OPTS="notebook --pylab inline" ./bin/pyspark {% endhighlight %} -IPython also works on a cluster or on multiple cores if you set the `MASTER` environment variable. +IPython also works on a cluster or on multiple cores if you set the `--master` flag. # Standalone Programs -PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `bin/spark-submit`. -The Quick Start guide includes a [complete example](quick-start.html#standalone-applications) of a standalone Python application. +PySpark can also be used from standalone Python scripts by creating a SparkContext in your script +and running the script using `bin/spark-submit`. The Quick Start guide includes a +[complete example](quick-start.html#standalone-applications) of a standalone Python application. Code dependencies can be deployed by passing .zip or .egg files in the `--py-files` option of `spark-submit`: @@ -138,6 +139,7 @@ You can set [configuration properties](configuration.html#spark-properties) by p {% highlight python %} from pyspark import SparkConf, SparkContext conf = (SparkConf() + .setMaster("local") .setAppName("My app") .set("spark.executor.memory", "1g")) sc = SparkContext(conf = conf) @@ -164,6 +166,6 @@ some example applications. PySpark also includes several sample programs in the [`examples/src/main/python` folder](https://github.com/apache/spark/tree/master/examples/src/main/python). You can run them by passing the files to `pyspark`; e.g.: - ./bin/spark-submit examples/src/main/python/wordcount.py local[2] README.md + ./bin/spark-submit examples/src/main/python/wordcount.py README.md -Each program prints usage help when run without arguments. +Each program prints usage help when run without the sufficient arguments. diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 01552dc1d449e..f0b46cd28b7aa 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -46,15 +46,15 @@ def update(i, vec, mat, ratings): return np.linalg.solve(XtX, Xty) if __name__ == "__main__": - if len(sys.argv) < 2: - print >> sys.stderr, "Usage: als " - exit(-1) - sc = SparkContext(sys.argv[1], "PythonALS", pyFiles=[realpath(__file__)]) - M = int(sys.argv[2]) if len(sys.argv) > 2 else 100 - U = int(sys.argv[3]) if len(sys.argv) > 3 else 500 - F = int(sys.argv[4]) if len(sys.argv) > 4 else 10 - ITERATIONS = int(sys.argv[5]) if len(sys.argv) > 5 else 5 - slices = int(sys.argv[6]) if len(sys.argv) > 6 else 2 + """ + Usage: als [M] [U] [F] [iterations] [slices]" + """ + sc = SparkContext(appName="PythonALS") + M = int(sys.argv[1]) if len(sys.argv) > 1 else 100 + U = int(sys.argv[2]) if len(sys.argv) > 2 else 500 + F = int(sys.argv[3]) if len(sys.argv) > 3 else 10 + ITERATIONS = int(sys.argv[4]) if len(sys.argv) > 4 else 5 + slices = int(sys.argv[5]) if len(sys.argv) > 5 else 2 print "Running ALS with M=%d, U=%d, F=%d, iters=%d, slices=%d\n" % \ (M, U, F, ITERATIONS, slices) diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index e3596488faf9e..fc16586c28a46 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -45,14 +45,14 @@ def closestPoint(p, centers): if __name__ == "__main__": - if len(sys.argv) < 5: - print >> sys.stderr, "Usage: kmeans " + if len(sys.argv) != 4: + print >> sys.stderr, "Usage: kmeans " exit(-1) - sc = SparkContext(sys.argv[1], "PythonKMeans") - lines = sc.textFile(sys.argv[2]) + sc = SparkContext(appName="PythonKMeans") + lines = sc.textFile(sys.argv[1]) data = lines.map(parseVector).cache() - K = int(sys.argv[3]) - convergeDist = float(sys.argv[4]) + K = int(sys.argv[2]) + convergeDist = float(sys.argv[3]) kPoints = data.takeSample(False, K, 1) tempDist = 1.0 diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index fe5373cf799b1..0f22d0b32319e 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -47,12 +47,12 @@ def readPointBatch(iterator): return [matrix] if __name__ == "__main__": - if len(sys.argv) != 4: - print >> sys.stderr, "Usage: logistic_regression " + if len(sys.argv) != 3: + print >> sys.stderr, "Usage: logistic_regression " exit(-1) - sc = SparkContext(sys.argv[1], "PythonLR", pyFiles=[realpath(__file__)]) - points = sc.textFile(sys.argv[2]).mapPartitions(readPointBatch).cache() - iterations = int(sys.argv[3]) + sc = SparkContext(appName="PythonLR") + points = sc.textFile(sys.argv[1]).mapPartitions(readPointBatch).cache() + iterations = int(sys.argv[2]) # Initialize w to a random value w = 2 * np.random.ranf(size=D) - 1 diff --git a/examples/src/main/python/mllib/kmeans.py b/examples/src/main/python/mllib/kmeans.py index dec82ff34fbac..b308132c9aeeb 100755 --- a/examples/src/main/python/mllib/kmeans.py +++ b/examples/src/main/python/mllib/kmeans.py @@ -33,12 +33,12 @@ def parseVector(line): if __name__ == "__main__": - if len(sys.argv) < 4: - print >> sys.stderr, "Usage: kmeans " + if len(sys.argv) != 3: + print >> sys.stderr, "Usage: kmeans " exit(-1) - sc = SparkContext(sys.argv[1], "KMeans") - lines = sc.textFile(sys.argv[2]) + sc = SparkContext(appName="KMeans") + lines = sc.textFile(sys.argv[1]) data = lines.map(parseVector) - k = int(sys.argv[3]) + k = int(sys.argv[2]) model = KMeans.train(data, k) print "Final centers: " + str(model.clusterCenters) diff --git a/examples/src/main/python/mllib/logistic_regression.py b/examples/src/main/python/mllib/logistic_regression.py index 8631051d00ff2..6e0f7a4ee5a81 100755 --- a/examples/src/main/python/mllib/logistic_regression.py +++ b/examples/src/main/python/mllib/logistic_regression.py @@ -39,12 +39,12 @@ def parsePoint(line): if __name__ == "__main__": - if len(sys.argv) != 4: - print >> sys.stderr, "Usage: logistic_regression " + if len(sys.argv) != 3: + print >> sys.stderr, "Usage: logistic_regression " exit(-1) - sc = SparkContext(sys.argv[1], "PythonLR") - points = sc.textFile(sys.argv[2]).map(parsePoint) - iterations = int(sys.argv[3]) + sc = SparkContext(appName="PythonLR") + points = sc.textFile(sys.argv[1]).map(parsePoint) + iterations = int(sys.argv[2]) model = LogisticRegressionWithSGD.train(points, iterations) print "Final weights: " + str(model.weights) print "Final intercept: " + str(model.intercept) diff --git a/examples/src/main/python/pagerank.py b/examples/src/main/python/pagerank.py index cd774cf3a319f..d350fa46fa49a 100755 --- a/examples/src/main/python/pagerank.py +++ b/examples/src/main/python/pagerank.py @@ -36,19 +36,19 @@ def parseNeighbors(urls): if __name__ == "__main__": - if len(sys.argv) < 3: - print >> sys.stderr, "Usage: pagerank " + if len(sys.argv) != 3: + print >> sys.stderr, "Usage: pagerank " exit(-1) # Initialize the spark context. - sc = SparkContext(sys.argv[1], "PythonPageRank") + sc = SparkContext(appName="PythonPageRank") # Loads in input file. It should be in format of: # URL neighbor URL # URL neighbor URL # URL neighbor URL # ... - lines = sc.textFile(sys.argv[2], 1) + lines = sc.textFile(sys.argv[1], 1) # Loads all URLs from input file and initialize their neighbors. links = lines.map(lambda urls: parseNeighbors(urls)).distinct().groupByKey().cache() @@ -57,7 +57,7 @@ def parseNeighbors(urls): ranks = links.map(lambda (url, neighbors): (url, 1.0)) # Calculates and updates URL ranks continuously using PageRank algorithm. - for iteration in xrange(int(sys.argv[3])): + for iteration in xrange(int(sys.argv[2])): # Calculates URL contributions to the rank of other URLs. contribs = links.join(ranks).flatMap(lambda (url, (urls, rank)): computeContribs(urls, rank)) diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index ab0645fc2f326..234720b55fa49 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -23,11 +23,11 @@ if __name__ == "__main__": - if len(sys.argv) == 1: - print >> sys.stderr, "Usage: pi []" - exit(-1) - sc = SparkContext(sys.argv[1], "PythonPi") - slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 + """ + Usage: pi [slices] + """ + sc = SparkContext(appName="PythonPi") + slices = int(sys.argv[1]) if len(sys.argv) > 1 else 2 n = 100000 * slices def f(_): x = random() * 2 - 1 diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py index 5de20a6d98f43..4913ee926aa03 100755 --- a/examples/src/main/python/sort.py +++ b/examples/src/main/python/sort.py @@ -21,11 +21,11 @@ if __name__ == "__main__": - if len(sys.argv) < 3: - print >> sys.stderr, "Usage: sort " + if len(sys.argv) != 2: + print >> sys.stderr, "Usage: sort " exit(-1) - sc = SparkContext(sys.argv[1], "PythonSort") - lines = sc.textFile(sys.argv[2], 1) + sc = SparkContext(appName="PythonSort") + lines = sc.textFile(sys.argv[1], 1) sortedCount = lines.flatMap(lambda x: x.split(' ')) \ .map(lambda x: (int(x), 1)) \ .sortByKey(lambda x: x) diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index 744cce6651607..8698369b13d84 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -36,11 +36,11 @@ def generateGraph(): if __name__ == "__main__": - if len(sys.argv) == 1: - print >> sys.stderr, "Usage: transitive_closure []" - exit(-1) - sc = SparkContext(sys.argv[1], "PythonTransitiveClosure") - slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 + """ + Usage: transitive_closure [slices] + """ + sc = SparkContext(appName="PythonTransitiveClosure") + slices = int(sys.argv[1]) if len(sys.argv) > 1 else 2 tc = sc.parallelize(generateGraph(), slices).cache() # Linear transitive closure: each round grows paths by one edge, diff --git a/examples/src/main/python/wordcount.py b/examples/src/main/python/wordcount.py index b9139b9d76520..dcc095fdd0ed9 100755 --- a/examples/src/main/python/wordcount.py +++ b/examples/src/main/python/wordcount.py @@ -22,11 +22,11 @@ if __name__ == "__main__": - if len(sys.argv) < 3: - print >> sys.stderr, "Usage: wordcount " + if len(sys.argv) != 2: + print >> sys.stderr, "Usage: wordcount " exit(-1) - sc = SparkContext(sys.argv[1], "PythonWordCount") - lines = sc.textFile(sys.argv[2], 1) + sc = SparkContext(appName="PythonWordCount") + lines = sc.textFile(sys.argv[1], 1) counts = lines.flatMap(lambda x: x.split(' ')) \ .map(lambda x: (x, 1)) \ .reduceByKey(add) From 442808a7482b81c8de887c901b424683da62022e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 16 May 2014 22:58:47 -0700 Subject: [PATCH 388/641] Make deprecation warning less severe Just a small change. I think it's good not to scare people who are using the old options. Author: Patrick Wendell Closes #810 from pwendell/warnings and squashes the following commits: cb8a311 [Patrick Wendell] Make deprecation warning less severe --- core/src/main/scala/org/apache/spark/SparkConf.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 800616622d7bf..8ce4b91cae8ae 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -238,10 +238,10 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { // Check for legacy configs sys.env.get("SPARK_JAVA_OPTS").foreach { value => - val error = + val warning = s""" |SPARK_JAVA_OPTS was detected (set to '$value'). - |This has undefined behavior when running on a cluster and is deprecated in Spark 1.0+. + |This is deprecated in Spark 1.0+. | |Please instead use: | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application @@ -249,7 +249,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { | - spark.executor.extraJavaOptions to set -X options for executors | - SPARK_DAEMON_JAVA_OPTS to set java options for standalone daemons (master or worker) """.stripMargin - logError(error) + logWarning(warning) for (key <- Seq(executorOptsKey, driverOptsKey)) { if (getOption(key).isDefined) { @@ -262,16 +262,16 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { } sys.env.get("SPARK_CLASSPATH").foreach { value => - val error = + val warning = s""" |SPARK_CLASSPATH was detected (set to '$value'). - | This has undefined behavior when running on a cluster and is deprecated in Spark 1.0+. + |This is deprecated in Spark 1.0+. | |Please instead use: | - ./spark-submit with --driver-class-path to augment the driver classpath | - spark.executor.extraClassPath to augment the executor classpath """.stripMargin - logError(error) + logWarning(warning) for (key <- Seq(executorClasspathKey, driverClassPathKey)) { if (getOption(key).isDefined) { From ebcd2d68897f423614d0629758240da97dca4a12 Mon Sep 17 00:00:00 2001 From: Neville Li Date: Sun, 18 May 2014 13:31:23 -0700 Subject: [PATCH 389/641] Fix spark-submit path in spark-shell & pyspark Author: Neville Li Closes #812 from nevillelyh/neville/v1.0 and squashes the following commits: 0dc33ed [Neville Li] Fix spark-submit path in pyspark becec64 [Neville Li] Fix spark-submit path in spark-shell --- bin/pyspark | 2 +- bin/spark-shell | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/bin/pyspark b/bin/pyspark index 9e1364e44c8c4..3908ffe79939b 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -27,7 +27,7 @@ SCALA_VERSION=2.10 if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then echo "Usage: ./bin/pyspark [options]" - ./bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 exit 0 fi diff --git a/bin/spark-shell b/bin/spark-shell index c158683ab3f99..850e9507ec38f 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -28,15 +28,15 @@ esac # Enter posix mode for bash set -o posix +## Global script variables +FWDIR="$(cd `dirname $0`/..; pwd)" + if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then echo "Usage: ./bin/spark-shell [options]" - ./bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 exit 0 fi -## Global script variables -FWDIR="$(cd `dirname $0`/..; pwd)" - function main(){ if $cygwin; then # Workaround for issue involving JLine and Cygwin From 4ce479324bdcf603806fc90b5b0f4968c6de690e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 18 May 2014 16:51:53 -0700 Subject: [PATCH 390/641] SPARK-1873: Add README.md file when making distributions Author: Patrick Wendell Closes #818 from pwendell/reamde and squashes the following commits: 4020b11 [Patrick Wendell] SPARK-1873: Add README.md file when making distributions --- make-distribution.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/make-distribution.sh b/make-distribution.sh index 7a08d6b9151c4..1c89027d68bed 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -191,6 +191,7 @@ fi mkdir "$DISTDIR"/conf cp "$FWDIR"/conf/*.template "$DISTDIR"/conf cp "$FWDIR"/conf/slaves "$DISTDIR"/conf +cp "$FWDIR/README.md" "$DISTDIR" cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/python" "$DISTDIR" cp -r "$FWDIR/sbin" "$DISTDIR" From df0aa8353ab6d3b19d838c6fa95a93a64948309f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 18 May 2014 17:00:57 -0700 Subject: [PATCH 391/641] [WIP][SPARK-1871][MLLIB] Improve MLlib guide for v1.0 Some improvements to MLlib guide: 1. [SPARK-1872] Update API links for unidoc. 2. [SPARK-1783] Added `page.displayTitle` to the global layout. If it is defined, use it instead of `page.title` for title display. 3. Add more Java/Python examples. Author: Xiangrui Meng Closes #816 from mengxr/mllib-doc and squashes the following commits: ec2e407 [Xiangrui Meng] format scala example for ALS cd9f40b [Xiangrui Meng] add a paragraph to summarize distributed matrix types 4617f04 [Xiangrui Meng] add python example to loadLibSVMFile and fix Java example d6509c2 [Xiangrui Meng] [SPARK-1783] update mllib titles 561fdc0 [Xiangrui Meng] add a displayTitle option to global layout 195d06f [Xiangrui Meng] add Java example for summary stats and minor fix 9f1ff89 [Xiangrui Meng] update java api links in mllib-basics 7dad18e [Xiangrui Meng] update java api links in NB 3a0f4a6 [Xiangrui Meng] api/pyspark -> api/python 35bdeb9 [Xiangrui Meng] api/mllib -> api/scala e4afaa8 [Xiangrui Meng] explicity state what might change --- docs/_layouts/global.html | 6 +- docs/mllib-basics.md | 125 +++++++++++++++++-------- docs/mllib-clustering.md | 5 +- docs/mllib-collaborative-filtering.md | 29 +++--- docs/mllib-decision-tree.md | 3 +- docs/mllib-dimensionality-reduction.md | 3 +- docs/mllib-guide.md | 19 ++-- docs/mllib-linear-methods.md | 21 +++-- docs/mllib-naive-bayes.md | 21 +++-- docs/mllib-optimization.md | 11 ++- 10 files changed, 153 insertions(+), 90 deletions(-) diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 8b543de574622..fb808129bb65d 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -114,7 +114,11 @@
      -

      {{ page.title }}

      + {% if page.displayTitle %} +

      {{ page.displayTitle }}

      + {% else %} +

      {{ page.title }}

      + {% endif %} {{ content }} diff --git a/docs/mllib-basics.md b/docs/mllib-basics.md index aa9321a547097..5796e16e8f99c 100644 --- a/docs/mllib-basics.md +++ b/docs/mllib-basics.md @@ -1,6 +1,7 @@ --- layout: global -title: MLlib - Basics +title: Basics - MLlib +displayTitle: MLlib - Basics --- * Table of contents @@ -26,11 +27,11 @@ of the vector.
      The base class of local vectors is -[`Vector`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector), and we provide two -implementations: [`DenseVector`](api/mllib/index.html#org.apache.spark.mllib.linalg.DenseVector) and -[`SparseVector`](api/mllib/index.html#org.apache.spark.mllib.linalg.SparseVector). We recommend +[`Vector`](api/scala/index.html#org.apache.spark.mllib.linalg.Vector), and we provide two +implementations: [`DenseVector`](api/scala/index.html#org.apache.spark.mllib.linalg.DenseVector) and +[`SparseVector`](api/scala/index.html#org.apache.spark.mllib.linalg.SparseVector). We recommend using the factory methods implemented in -[`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) to create local vectors. +[`Vectors`](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) to create local vectors. {% highlight scala %} import org.apache.spark.mllib.linalg.{Vector, Vectors} @@ -53,11 +54,11 @@ Scala imports `scala.collection.immutable.Vector` by default, so you have to imp
      The base class of local vectors is -[`Vector`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector), and we provide two -implementations: [`DenseVector`](api/mllib/index.html#org.apache.spark.mllib.linalg.DenseVector) and -[`SparseVector`](api/mllib/index.html#org.apache.spark.mllib.linalg.SparseVector). We recommend +[`Vector`](api/java/org/apache/spark/mllib/linalg/Vector.html), and we provide two +implementations: [`DenseVector`](api/java/org/apache/spark/mllib/linalg/DenseVector.html) and +[`SparseVector`](api/java/org/apache/spark/mllib/linalg/SparseVector.html). We recommend using the factory methods implemented in -[`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) to create local vectors. +[`Vectors`](api/java/org/apache/spark/mllib/linalg/Vector.html) to create local vectors. {% highlight java %} import org.apache.spark.mllib.linalg.Vector; @@ -78,13 +79,13 @@ MLlib recognizes the following types as dense vectors: and the following as sparse vectors: -* MLlib's [`SparseVector`](api/pyspark/pyspark.mllib.linalg.SparseVector-class.html). +* MLlib's [`SparseVector`](api/python/pyspark.mllib.linalg.SparseVector-class.html). * SciPy's [`csc_matrix`](http://docs.scipy.org/doc/scipy/reference/generated/scipy.sparse.csc_matrix.html#scipy.sparse.csc_matrix) with a single column We recommend using NumPy arrays over lists for efficiency, and using the factory methods implemented -in [`Vectors`](api/pyspark/pyspark.mllib.linalg.Vectors-class.html) to create sparse vectors. +in [`Vectors`](api/python/pyspark.mllib.linalg.Vectors-class.html) to create sparse vectors. {% highlight python %} import numpy as np @@ -117,7 +118,7 @@ For multiclass classification, labels should be class indices staring from zero:
      A labeled point is represented by the case class -[`LabeledPoint`](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint). +[`LabeledPoint`](api/scala/index.html#org.apache.spark.mllib.regression.LabeledPoint). {% highlight scala %} import org.apache.spark.mllib.linalg.Vectors @@ -134,7 +135,7 @@ val neg = LabeledPoint(0.0, Vectors.sparse(3, Array(0, 2), Array(1.0, 3.0)))
      A labeled point is represented by -[`LabeledPoint`](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint). +[`LabeledPoint`](api/java/org/apache/spark/mllib/regression/LabeledPoint.html). {% highlight java %} import org.apache.spark.mllib.linalg.Vectors; @@ -151,7 +152,7 @@ LabeledPoint neg = new LabeledPoint(1.0, Vectors.sparse(3, new int[] {0, 2}, new
      A labeled point is represented by -[`LabeledPoint`](api/pyspark/pyspark.mllib.regression.LabeledPoint-class.html). +[`LabeledPoint`](api/python/pyspark.mllib.regression.LabeledPoint-class.html). {% highlight python %} from pyspark.mllib.linalg import SparseVector @@ -184,7 +185,7 @@ After loading, the feature indices are converted to zero-based.
      -[`MLUtils.loadLibSVMFile`](api/mllib/index.html#org.apache.spark.mllib.util.MLUtils$) reads training +[`MLUtils.loadLibSVMFile`](api/scala/index.html#org.apache.spark.mllib.util.MLUtils$) reads training examples stored in LIBSVM format. {% highlight scala %} @@ -192,20 +193,32 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD -val training: RDD[LabeledPoint] = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") +val examples: RDD[LabeledPoint] = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") {% endhighlight %}
      -[`MLUtils.loadLibSVMFile`](api/mllib/index.html#org.apache.spark.mllib.util.MLUtils$) reads training +[`MLUtils.loadLibSVMFile`](api/java/org/apache/spark/mllib/util/MLUtils.html) reads training examples stored in LIBSVM format. {% highlight java %} import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.mllib.util.MLUtils; -import org.apache.spark.rdd.RDDimport; +import org.apache.spark.api.java.JavaRDD; + +JavaRDD examples = + MLUtils.loadLibSVMFile(jsc.sc(), "mllib/data/sample_libsvm_data.txt").toJavaRDD(); +{% endhighlight %} +
      + +
      +[`MLUtils.loadLibSVMFile`](api/python/pyspark.mllib.util.MLUtils-class.html) reads training +examples stored in LIBSVM format. -RDD training = MLUtils.loadLibSVMFile(jsc, "mllib/data/sample_libsvm_data.txt"); +{% highlight python %} +from pyspark.mllib.util import MLUtils + +examples = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") {% endhighlight %}
      @@ -227,10 +240,10 @@ We are going to add sparse matrix in the next release.
      The base class of local matrices is -[`Matrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.Matrix), and we provide one -implementation: [`DenseMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.DenseMatrix). +[`Matrix`](api/scala/index.html#org.apache.spark.mllib.linalg.Matrix), and we provide one +implementation: [`DenseMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.DenseMatrix). Sparse matrix will be added in the next release. We recommend using the factory methods implemented -in [`Matrices`](api/mllib/index.html#org.apache.spark.mllib.linalg.Matrices) to create local +in [`Matrices`](api/scala/index.html#org.apache.spark.mllib.linalg.Matrices) to create local matrices. {% highlight scala %} @@ -244,10 +257,10 @@ val dm: Matrix = Matrices.dense(3, 2, Array(1.0, 3.0, 5.0, 2.0, 4.0, 6.0))
      The base class of local matrices is -[`Matrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.Matrix), and we provide one -implementation: [`DenseMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.DenseMatrix). +[`Matrix`](api/java/org/apache/spark/mllib/linalg/Matrix.html), and we provide one +implementation: [`DenseMatrix`](api/java/org/apache/spark/mllib/linalg/DenseMatrix.html). Sparse matrix will be added in the next release. We recommend using the factory methods implemented -in [`Matrices`](api/mllib/index.html#org.apache.spark.mllib.linalg.Matrices) to create local +in [`Matrices`](api/java/org/apache/spark/mllib/linalg/Matrices.html) to create local matrices. {% highlight java %} @@ -269,6 +282,15 @@ and distributed matrices. Converting a distributed matrix to a different format global shuffle, which is quite expensive. We implemented three types of distributed matrices in this release and will add more types in the future. +The basic type is called `RowMatrix`. A `RowMatrix` is a row-oriented distributed +matrix without meaningful row indices, e.g., a collection of feature vectors. +It is backed by an RDD of its rows, where each row is a local vector. +We assume that the number of columns is not huge for a `RowMatrix`. +An `IndexedRowMatrix` is similar to a `RowMatrix` but with row indices, +which can be used for identifying rows and joins. +A `CoordinateMatrix` is a distributed matrix stored in [coordinate list (COO)](https://en.wikipedia.org/wiki/Sparse_matrix) format, +backed by an RDD of its entries. + ***Note*** The underlying RDDs of a distributed matrix must be deterministic, because we cache the matrix size. @@ -284,7 +306,7 @@ limited by the integer range but it should be much smaller in practice.
      -A [`RowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) can be +A [`RowMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) can be created from an `RDD[Vector]` instance. Then we can compute its column summary statistics. {% highlight scala %} @@ -303,7 +325,7 @@ val n = mat.numCols()
      -A [`RowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) can be +A [`RowMatrix`](api/java/org/apache/spark/mllib/linalg/distributed/RowMatrix.html) can be created from a `JavaRDD` instance. Then we can compute its column summary statistics. {% highlight java %} @@ -333,8 +355,8 @@ which could be faster if the rows are sparse.
      -`RowMatrix#computeColumnSummaryStatistics` returns an instance of -[`MultivariateStatisticalSummary`](api/mllib/index.html#org.apache.spark.mllib.stat.MultivariateStatisticalSummary), +[`RowMatrix#computeColumnSummaryStatistics`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) returns an instance of +[`MultivariateStatisticalSummary`](api/scala/index.html#org.apache.spark.mllib.stat.MultivariateStatisticalSummary), which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the total count. @@ -355,6 +377,31 @@ println(summary.numNonzeros) // number of nonzeros in each column val cov: Matrix = mat.computeCovariance() {% endhighlight %}
      + +
      + +[`RowMatrix#computeColumnSummaryStatistics`](api/java/org/apache/spark/mllib/linalg/distributed/RowMatrix.html#computeColumnSummaryStatistics()) returns an instance of +[`MultivariateStatisticalSummary`](api/java/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.html), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight java %} +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary; + +RowMatrix mat = ... // a RowMatrix + +// Compute column summary statistics. +MultivariateStatisticalSummary summary = mat.computeColumnSummaryStatistics(); +System.out.println(summary.mean()); // a dense vector containing the mean value for each column +System.out.println(summary.variance()); // column-wise variance +System.out.println(summary.numNonzeros()); // number of nonzeros in each column + +// Compute the covariance matrix. +Matrix cov = mat.computeCovariance(); +{% endhighlight %} +
      ### IndexedRowMatrix @@ -366,9 +413,9 @@ an RDD of indexed rows, which each row is represented by its index (long-typed)
      An -[`IndexedRowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix) +[`IndexedRowMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix) can be created from an `RDD[IndexedRow]` instance, where -[`IndexedRow`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRow) is a +[`IndexedRow`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRow) is a wrapper over `(Long, Vector)`. An `IndexedRowMatrix` can be converted to a `RowMatrix` by dropping its row indices. @@ -391,9 +438,9 @@ val rowMat: RowMatrix = mat.toRowMatrix()
      An -[`IndexedRowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix) +[`IndexedRowMatrix`](api/java/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.html) can be created from an `JavaRDD` instance, where -[`IndexedRow`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRow) is a +[`IndexedRow`](api/java/org/apache/spark/mllib/linalg/distributed/IndexedRow.html) is a wrapper over `(long, Vector)`. An `IndexedRowMatrix` can be converted to a `RowMatrix` by dropping its row indices. @@ -427,9 +474,9 @@ dimensions of the matrix are huge and the matrix is very sparse.
      A -[`CoordinateMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.CoordinateMatrix) +[`CoordinateMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.CoordinateMatrix) can be created from an `RDD[MatrixEntry]` instance, where -[`MatrixEntry`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.MatrixEntry) is a +[`MatrixEntry`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.MatrixEntry) is a wrapper over `(Long, Long, Double)`. A `CoordinateMatrix` can be converted to a `IndexedRowMatrix` with sparse rows by calling `toIndexedRowMatrix`. In this release, we do not provide other computation for `CoordinateMatrix`. @@ -453,13 +500,13 @@ val indexedRowMatrix = mat.toIndexedRowMatrix()
      A -[`CoordinateMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.CoordinateMatrix) +[`CoordinateMatrix`](api/java/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.html) can be created from a `JavaRDD` instance, where -[`MatrixEntry`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.MatrixEntry) is a +[`MatrixEntry`](api/java/org/apache/spark/mllib/linalg/distributed/MatrixEntry.html) is a wrapper over `(long, long, double)`. A `CoordinateMatrix` can be converted to a `IndexedRowMatrix` with sparse rows by calling `toIndexedRowMatrix`. -{% highlight scala %} +{% highlight java %} import org.apache.spark.api.java.JavaRDD; import org.apache.spark.mllib.linalg.distributed.CoordinateMatrix; import org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix; @@ -467,7 +514,7 @@ import org.apache.spark.mllib.linalg.distributed.MatrixEntry; JavaRDD entries = ... // a JavaRDD of matrix entries // Create a CoordinateMatrix from a JavaRDD. -CoordinateMatrix mat = new CoordinateMatrix(entries); +CoordinateMatrix mat = new CoordinateMatrix(entries.rdd()); // Get its size. long m = mat.numRows(); diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 276868fa8490d..429cdf8d40cec 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -1,6 +1,7 @@ --- layout: global -title: MLlib - Clustering +title: Clustering - MLlib +displayTitle: MLlib - Clustering --- * Table of contents @@ -40,7 +41,7 @@ a given dataset, the algorithm returns the best clustering result). Following code snippets can be executed in `spark-shell`. In the following example after loading and parsing data, we use the -[`KMeans`](api/mllib/index.html#org.apache.spark.mllib.clustering.KMeans) object to cluster the data +[`KMeans`](api/scala/index.html#org.apache.spark.mllib.clustering.KMeans) object to cluster the data into two clusters. The number of desired clusters is passed to the algorithm. We then compute Within Set Sum of Squared Error (WSSSE). You can reduce this error measure by increasing *k*. In fact the optimal *k* is usually one where there is an "elbow" in the WSSSE graph. diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index f486c56e55907..d51002f015670 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -1,6 +1,7 @@ --- layout: global -title: MLlib - Collaborative Filtering +title: Collaborative Filtering - MLlib +displayTitle: MLlib - Collaborative Filtering --- * Table of contents @@ -48,7 +49,7 @@ user for an item.
      In the following example we load rating data. Each row consists of a user, a product and a rating. -We use the default [ALS.train()](api/mllib/index.html#org.apache.spark.mllib.recommendation.ALS$) +We use the default [ALS.train()](api/scala/index.html#org.apache.spark.mllib.recommendation.ALS$) method which assumes ratings are explicit. We evaluate the recommendation model by measuring the Mean Squared Error of rating prediction. @@ -58,9 +59,9 @@ import org.apache.spark.mllib.recommendation.Rating // Load and parse the data val data = sc.textFile("mllib/data/als/test.data") -val ratings = data.map(_.split(',') match { - case Array(user, item, rate) => Rating(user.toInt, item.toInt, rate.toDouble) -}) +val ratings = data.map(_.split(',') match { case Array(user, item, rate) => + Rating(user.toInt, item.toInt, rate.toDouble) + }) // Build the recommendation model using ALS val rank = 10 @@ -68,15 +69,19 @@ val numIterations = 20 val model = ALS.train(ratings, rank, numIterations, 0.01) // Evaluate the model on rating data -val usersProducts = ratings.map{ case Rating(user, product, rate) => (user, product)} -val predictions = model.predict(usersProducts).map{ - case Rating(user, product, rate) => ((user, product), rate) +val usersProducts = ratings.map { case Rating(user, product, rate) => + (user, product) } -val ratesAndPreds = ratings.map{ - case Rating(user, product, rate) => ((user, product), rate) +val predictions = + model.predict(usersProducts).map { case Rating(user, product, rate) => + ((user, product), rate) + } +val ratesAndPreds = ratings.map { case Rating(user, product, rate) => + ((user, product), rate) }.join(predictions) -val MSE = ratesAndPreds.map{ - case ((user, product), (r1, r2)) => math.pow((r1- r2), 2) +val MSE = ratesAndPreds.map { case ((user, product), (r1, r2)) => + val err = (r1 - r2) + err * err }.mean() println("Mean Squared Error = " + MSE) {% endhighlight %} diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index acf0feff42a8d..3002a66a4fdb3 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -1,6 +1,7 @@ --- layout: global -title: MLlib - Decision Tree +title: Decision Tree - MLlib +displayTitle: MLlib - Decision Tree --- * Table of contents diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md index ab24663cfe258..e3608075fbb13 100644 --- a/docs/mllib-dimensionality-reduction.md +++ b/docs/mllib-dimensionality-reduction.md @@ -1,6 +1,7 @@ --- layout: global -title: MLlib - Dimensionality Reduction +title: Dimensionality Reduction - MLlib +displayTitle: MLlib - Dimensionality Reduction --- * Table of contents diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 842ca5c8c6d8a..640ca83085387 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -27,8 +27,9 @@ filtering, dimensionality reduction, as well as underlying optimization primitiv * stochastic gradient descent * limited-memory BFGS (L-BFGS) -MLlib is currently a *beta* component under active development. -The APIs may change in the future releases, and we will provide migration guide between releases. +MLlib is a new component under active development. +The APIs marked `Experimental`/`DeveloperApi` may change in future releases, +and we will provide migration guide between releases. ## Dependencies @@ -61,9 +62,9 @@ take advantage of sparsity in both storage and computation.
      We used to represent a feature vector by `Array[Double]`, which is replaced by -[`Vector`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) in v1.0. Algorithms that used +[`Vector`](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) in v1.0. Algorithms that used to accept `RDD[Array[Double]]` now take -`RDD[Vector]`. [`LabeledPoint`](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint) +`RDD[Vector]`. [`LabeledPoint`](api/scala/index.html#org.apache.spark.mllib.regression.LabeledPoint) is now a wrapper of `(Double, Vector)` instead of `(Double, Array[Double])`. Converting `Array[Double]` to `Vector` is straightforward: @@ -74,7 +75,7 @@ val array: Array[Double] = ... // a double array val vector: Vector = Vectors.dense(array) // a dense vector {% endhighlight %} -[`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to create sparse vectors. +[`Vectors`](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to create sparse vectors. *Note*. Scala imports `scala.collection.immutable.Vector` by default, so you have to import `org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`. @@ -83,9 +84,9 @@ val vector: Vector = Vectors.dense(array) // a dense vector
      We used to represent a feature vector by `double[]`, which is replaced by -[`Vector`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) in v1.0. Algorithms that used +[`Vector`](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) in v1.0. Algorithms that used to accept `RDD` now take -`RDD`. [`LabeledPoint`](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint) +`RDD`. [`LabeledPoint`](api/scala/index.html#org.apache.spark.mllib.regression.LabeledPoint) is now a wrapper of `(double, Vector)` instead of `(double, double[])`. Converting `double[]` to `Vector` is straightforward: @@ -97,7 +98,7 @@ double[] array = ... // a double array Vector vector = Vectors.dense(array); // a dense vector {% endhighlight %} -[`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to +[`Vectors`](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to create sparse vectors.
      @@ -106,7 +107,7 @@ create sparse vectors. We used to represent a labeled feature vector in a NumPy array, where the first entry corresponds to the label and the rest are features. This representation is replaced by class -[`LabeledPoint`](api/pyspark/pyspark.mllib.regression.LabeledPoint-class.html), which takes both +[`LabeledPoint`](api/python/pyspark.mllib.regression.LabeledPoint-class.html), which takes both dense and sparse feature vectors. {% highlight python %} diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index eff617d8641e2..4dfbebbcd04b7 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -1,6 +1,7 @@ --- layout: global -title: MLlib - Linear Methods +title: Linear Methods - MLlib +displayTitle: MLlib - Linear Methods --- * Table of contents @@ -233,7 +234,7 @@ val modelL1 = svmAlg.run(training) {% endhighlight %} Similarly, you can use replace `SVMWithSGD` by -[`LogisticRegressionWithSGD`](api/mllib/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD). +[`LogisticRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD).
      @@ -328,8 +329,8 @@ println("training Mean Squared Error = " + MSE) {% endhighlight %} Similarly you can use -[`RidgeRegressionWithSGD`](api/mllib/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) -and [`LassoWithSGD`](api/mllib/index.html#org.apache.spark.mllib.regression.LassoWithSGD). +[`RidgeRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) +and [`LassoWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.LassoWithSGD).
      @@ -380,11 +381,11 @@ all three possible regularizations (none, L1 or L2). Algorithms are all implemented in Scala: -* [SVMWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.SVMWithSGD) -* [LogisticRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) -* [LinearRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD) -* [RidgeRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) -* [LassoWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.LassoWithSGD) +* [SVMWithSGD](api/scala/index.html#org.apache.spark.mllib.classification.SVMWithSGD) +* [LogisticRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) +* [LinearRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD) +* [RidgeRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) +* [LassoWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.LassoWithSGD) Python calls the Scala implementation via -[PythonMLLibAPI](api/mllib/index.html#org.apache.spark.mllib.api.python.PythonMLLibAPI). +[PythonMLLibAPI](api/scala/index.html#org.apache.spark.mllib.api.python.PythonMLLibAPI). diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index c47508b7daa2d..4b3a7cab32118 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -1,6 +1,7 @@ --- layout: global -title: MLlib - Naive Bayes +title: Naive Bayes - MLlib +displayTitle: MLlib - Naive Bayes --- Naive Bayes is a simple multiclass classification algorithm with the assumption of independence @@ -27,11 +28,11 @@ sparsity. Since the training data is only used once, it is not necessary to cach
      -[NaiveBayes](api/mllib/index.html#org.apache.spark.mllib.classification.NaiveBayes$) implements +[NaiveBayes](api/scala/index.html#org.apache.spark.mllib.classification.NaiveBayes$) implements multinomial naive Bayes. It takes an RDD of -[LabeledPoint](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint) and an optional +[LabeledPoint](api/scala/index.html#org.apache.spark.mllib.regression.LabeledPoint) and an optional smoothing parameter `lambda` as input, and output a -[NaiveBayesModel](api/mllib/index.html#org.apache.spark.mllib.classification.NaiveBayesModel), which +[NaiveBayesModel](api/scala/index.html#org.apache.spark.mllib.classification.NaiveBayesModel), which can be used for evaluation and prediction. {% highlight scala %} @@ -59,11 +60,11 @@ val accuracy = 1.0 * predictionAndLabel.filter(x => x._1 == x._2).count() / test
      -[NaiveBayes](api/mllib/index.html#org.apache.spark.mllib.classification.NaiveBayes$) implements +[NaiveBayes](api/java/org/apache/spark/mllib/classification/NaiveBayes.html) implements multinomial naive Bayes. It takes a Scala RDD of -[LabeledPoint](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint) and an +[LabeledPoint](api/java/org/apache/spark/mllib/regression/LabeledPoint.html) and an optionally smoothing parameter `lambda` as input, and output a -[NaiveBayesModel](api/mllib/index.html#org.apache.spark.mllib.classification.NaiveBayesModel), which +[NaiveBayesModel](api/java/org/apache/spark/mllib/classification/NaiveBayesModel.html), which can be used for evaluation and prediction. {% highlight java %} @@ -102,11 +103,11 @@ double accuracy = 1.0 * predictionAndLabel.filter(new Function -[NaiveBayes](api/pyspark/pyspark.mllib.classification.NaiveBayes-class.html) implements multinomial +[NaiveBayes](api/python/pyspark.mllib.classification.NaiveBayes-class.html) implements multinomial naive Bayes. It takes an RDD of -[LabeledPoint](api/pyspark/pyspark.mllib.regression.LabeledPoint-class.html) and an optionally +[LabeledPoint](api/python/pyspark.mllib.regression.LabeledPoint-class.html) and an optionally smoothing parameter `lambda` as input, and output a -[NaiveBayesModel](api/pyspark/pyspark.mllib.classification.NaiveBayesModel-class.html), which can be +[NaiveBayesModel](api/python/pyspark.mllib.classification.NaiveBayesModel-class.html), which can be used for evaluation and prediction. diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index aa0dec2130593..a22980d03a2f0 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -1,6 +1,7 @@ --- layout: global -title: MLlib - Optimization +title: Optimization - MLlib +displayTitle: MLlib - Optimization --- * Table of contents @@ -170,17 +171,17 @@ each iteration, to compute the gradient direction. Available algorithms for gradient descent: -* [GradientDescent.runMiniBatchSGD](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) +* [GradientDescent.runMiniBatchSGD](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) ### L-BFGS L-BFGS is currently only a low-level optimization primitive in `MLlib`. If you want to use L-BFGS in various ML algorithms such as Linear Regression, and Logistic Regression, you have to pass the gradient of objective function, and updater into optimizer yourself instead of using the training APIs like -[LogisticRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD). +[LogisticRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD). See the example below. It will be addressed in the next release. The L1 regularization by using -[L1Updater](api/mllib/index.html#org.apache.spark.mllib.optimization.L1Updater) will not work since the +[L1Updater](api/scala/index.html#org.apache.spark.mllib.optimization.L1Updater) will not work since the soft-thresholding logic in L1Updater is designed for gradient descent. See the developer's note. The L-BFGS method @@ -274,4 +275,4 @@ the actual gradient descent step. However, we're able to take the gradient and loss of objective function of regularization for L-BFGS by ignoring the part of logic only for gradient decent such as adaptive step size stuff. We will refactorize this into regularizer to replace updater to separate the logic between -regularization and step update later. \ No newline at end of file +regularization and step update later. From 7b70a7071894dd90ea1d0091542b3e13e7ef8d3a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 19 May 2014 15:02:35 -0700 Subject: [PATCH 392/641] [SPARK-1876] Windows fixes to deal with latest distribution layout changes - Look for JARs in the right place - Launch examples the same way as on Unix - Load datanucleus JARs if they exist - Don't attempt to parse local paths as URIs in SparkSubmit, since paths with C:\ are not valid URIs - Also fixed POM exclusion rules for datanucleus (it wasn't properly excluding it, whereas SBT was) Author: Matei Zaharia Closes #819 from mateiz/win-fixes and squashes the following commits: d558f96 [Matei Zaharia] Fix comment 228577b [Matei Zaharia] Review comments d3b71c7 [Matei Zaharia] Properly exclude datanucleus files in Maven assembly 144af84 [Matei Zaharia] Update Windows scripts to match latest binary package layout --- README.md | 7 +-- assembly/pom.xml | 2 +- bin/compute-classpath.cmd | 24 ++++++++- bin/run-example | 23 ++++----- bin/run-example2.cmd | 51 ++++++++++++++----- bin/spark-class2.cmd | 2 + .../org/apache/spark/deploy/SparkSubmit.scala | 2 +- 7 files changed, 81 insertions(+), 30 deletions(-) diff --git a/README.md b/README.md index 9c2e32b90f162..6211a5889a3f5 100644 --- a/README.md +++ b/README.md @@ -9,13 +9,14 @@ You can find the latest Spark documentation, including a programming guide, on the project webpage at . This README file only contains basic setup instructions. - ## Building Spark Spark is built on Scala 2.10. To build Spark and its example programs, run: ./sbt/sbt assembly +(You do not need to do this if you downloaded a pre-built package.) + ## Interactive Scala Shell The easiest way to start using Spark is through the Scala shell: @@ -41,9 +42,9 @@ And run the following command, which should also return 1000: Spark also comes with several sample programs in the `examples` directory. To run one of them, use `./bin/run-example [params]`. For example: - ./bin/run-example org.apache.spark.examples.SparkLR + ./bin/run-example SparkPi -will run the Logistic Regression example locally. +will run the Pi example locally. You can set the MASTER environment variable when running examples to submit examples to a cluster. This can be a mesos:// or spark:// URL, diff --git a/assembly/pom.xml b/assembly/pom.xml index abd8935339992..963357b9ab167 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -96,7 +96,7 @@ *:* - org.datanucleus:* + org/datanucleus/** META-INF/*.SF META-INF/*.DSA META-INF/*.RSA diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 065553eb31939..58710cd1bd548 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -20,6 +20,13 @@ rem rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run" rem script and the ExecutorRunner in standalone cluster mode. +rem If we're called from spark-class2.cmd, it already set enabledelayedexpansion and setting +rem it here would stop us from affecting its copy of the CLASSPATH variable; otherwise we +rem need to set it here because we use !datanucleus_jars! below. +if "%DONT_PRINT_CLASSPATH%"=="1" goto skip_delayed_expansion +setlocal enabledelayedexpansion +:skip_delayed_expansion + set SCALA_VERSION=2.10 rem Figure out where the Spark framework is installed @@ -31,7 +38,7 @@ if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Build up classpath set CLASSPATH=%FWDIR%conf if exist "%FWDIR%RELEASE" ( - for %%d in ("%FWDIR%jars\spark-assembly*.jar") do ( + for %%d in ("%FWDIR%lib\spark-assembly*.jar") do ( set ASSEMBLY_JAR=%%d ) ) else ( @@ -42,6 +49,21 @@ if exist "%FWDIR%RELEASE" ( set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% +rem When Hive support is needed, Datanucleus jars must be included on the classpath. +rem Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. +rem Both sbt and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is +rem built with Hive, so look for them there. +if exist "%FWDIR%RELEASE" ( + set datanucleus_dir=%FWDIR%lib +) else ( + set datanucleus_dir=%FWDIR%lib_managed\jars +) +set "datanucleus_jars=" +for %%d in ("%datanucleus_dir%\datanucleus-*.jar") do ( + set datanucleus_jars=!datanucleus_jars!;%%d +) +set CLASSPATH=%CLASSPATH%;%datanucleus_jars% + set SPARK_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\classes set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\classes set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\classes diff --git a/bin/run-example b/bin/run-example index 146951ac0ee56..7caab31daef39 100755 --- a/bin/run-example +++ b/bin/run-example @@ -23,6 +23,16 @@ FWDIR="$(cd `dirname $0`/..; pwd)" export SPARK_HOME="$FWDIR" EXAMPLES_DIR="$FWDIR"/examples +if [ -n "$1" ]; then + EXAMPLE_CLASS="$1" + shift +else + echo "Usage: ./bin/run-example [example-args]" + echo " - set MASTER=XX to use a specific master" + echo " - can use abbreviated example class name (e.g. SparkPi, mllib.LinearRegression)" + exit 1 +fi + if [ -f "$FWDIR/RELEASE" ]; then export SPARK_EXAMPLES_JAR=`ls "$FWDIR"/lib/spark-examples-*hadoop*.jar` elif [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar ]; then @@ -37,17 +47,6 @@ fi EXAMPLE_MASTER=${MASTER:-"local[*]"} -if [ -n "$1" ]; then - EXAMPLE_CLASS="$1" - shift -else - echo "usage: ./bin/run-example [example-args]" - echo " - set MASTER=XX to use a specific master" - echo " - can use abbreviated example class name (e.g. SparkPi, mllib.MovieLensALS)" - echo - exit -1 -fi - if [[ ! $EXAMPLE_CLASS == org.apache.spark.examples* ]]; then EXAMPLE_CLASS="org.apache.spark.examples.$EXAMPLE_CLASS" fi @@ -55,5 +54,5 @@ fi ./bin/spark-submit \ --master $EXAMPLE_MASTER \ --class $EXAMPLE_CLASS \ - $SPARK_EXAMPLES_JAR \ + "$SPARK_EXAMPLES_JAR" \ "$@" diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd index 40abb9af74246..eadedd7fa61ff 100644 --- a/bin/run-example2.cmd +++ b/bin/run-example2.cmd @@ -30,7 +30,9 @@ if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Test that an argument was given if not "x%1"=="x" goto arg_given - echo Usage: run-example ^ [^] + echo Usage: run-example ^ [example-args] + echo - set MASTER=XX to use a specific master + echo - can use abbreviated example class name (e.g. SparkPi, mllib.LinearRegression) goto exit :arg_given @@ -38,8 +40,14 @@ set EXAMPLES_DIR=%FWDIR%examples rem Figure out the JAR file that our examples were packaged into. set SPARK_EXAMPLES_JAR= -for %%d in ("%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\spark-examples*assembly*.jar") do ( - set SPARK_EXAMPLES_JAR=%%d +if exist "%FWDIR%RELEASE" ( + for %%d in ("%FWDIR%lib\spark-examples*.jar") do ( + set SPARK_EXAMPLES_JAR=%%d + ) +) else ( + for %%d in ("%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\spark-examples*.jar") do ( + set SPARK_EXAMPLES_JAR=%%d + ) ) if "x%SPARK_EXAMPLES_JAR%"=="x" ( echo Failed to find Spark examples assembly JAR. @@ -47,15 +55,34 @@ if "x%SPARK_EXAMPLES_JAR%"=="x" ( goto exit ) -rem Compute Spark classpath using external script -set DONT_PRINT_CLASSPATH=1 -call "%FWDIR%bin\compute-classpath.cmd" -set DONT_PRINT_CLASSPATH=0 -set CLASSPATH=%SPARK_EXAMPLES_JAR%;%CLASSPATH% +rem Set master from MASTER environment variable if given +if "x%MASTER%"=="x" ( + set EXAMPLE_MASTER=local[*] +) else ( + set EXAMPLE_MASTER=%MASTER% +) + +rem If the EXAMPLE_CLASS does not start with org.apache.spark.examples, add that +set EXAMPLE_CLASS=%1 +set PREFIX=%EXAMPLE_CLASS:~0,25% +if not %PREFIX%==org.apache.spark.examples ( + set EXAMPLE_CLASS=org.apache.spark.examples.%EXAMPLE_CLASS% +) + +rem Get the tail of the argument list, to skip the first one. This is surprisingly +rem complicated on Windows. +set "ARGS=" +:top +shift +if "%~1" neq "" ( + set ARGS=%ARGS% "%~1" + goto :top +) +if defined ARGS set ARGS=%ARGS:~1% -rem Figure out where java is. -set RUNNER=java -if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java +call "%FWDIR%bin\spark-submit.cmd" ^ + --master %EXAMPLE_MASTER% ^ + --class %EXAMPLE_CLASS% ^ + "%SPARK_EXAMPLES_JAR%" %ARGS% -"%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %* :exit diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index 4302c1b6b7ff4..266edd9fa9835 100755 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -17,6 +17,8 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem +setlocal enabledelayedexpansion + set SCALA_VERSION=2.10 rem Figure out where the Spark framework is installed diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index a99b2176e2b5e..c54331c00fab8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -299,7 +299,7 @@ object SparkSubmit { } private def addJarToClasspath(localJar: String, loader: ExecutorURLClassLoader) { - val localJarFile = new File(new URI(localJar).getPath) + val localJarFile = new File(localJar) if (!localJarFile.exists()) { printWarning(s"Jar $localJar does not exist, skipping.") } From 1811ba8ccb580979aa2e12019e6a82805f09ab53 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 19 May 2014 16:41:31 -0700 Subject: [PATCH 393/641] SPARK-1878: Fix the incorrect initialization order JIRA: https://issues.apache.org/jira/browse/SPARK-1878 Author: zsxwing Closes #822 from zsxwing/SPARK-1878 and squashes the following commits: 4a47e27 [zsxwing] SPARK-1878: Fix the incorrect initialization order --- .../spark/streaming/api/java/JavaStreamingContext.scala | 6 +++--- .../test/java/org/apache/spark/streaming/JavaAPISuite.java | 4 ++++ 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 75a3e9334e6d5..18605cac7006c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -142,12 +142,12 @@ class JavaStreamingContext(val ssc: StreamingContext) { */ def this(path: String, hadoopConf: Configuration) = this(new StreamingContext(path, hadoopConf)) - @deprecated("use sparkContext", "0.9.0") - val sc: JavaSparkContext = sparkContext - /** The underlying SparkContext */ val sparkContext = new JavaSparkContext(ssc.sc) + @deprecated("use sparkContext", "0.9.0") + val sc: JavaSparkContext = sparkContext + /** * Create an input stream from network source hostname:port. Data is received using * a TCP socket and the receive bytes is interpreted as UTF8 encoded \n delimited diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index ce58cb12a4564..4efeb8dfbe1ad 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -55,6 +55,10 @@ public void equalIterable(Iterable a, Iterable b) { equalIterator(a.iterator(), b.iterator()); } + @Test + public void testInitialization() { + Assert.assertNotNull(ssc.sc()); + } @SuppressWarnings("unchecked") @Test From 5af99d7617ba3b9fbfdb345ef9571b7dd41f45a1 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 19 May 2014 18:42:28 -0700 Subject: [PATCH 394/641] SPARK-1879. Increase MaxPermSize since some of our builds have many classes See https://issues.apache.org/jira/browse/SPARK-1879 -- builds with Hadoop2 and Hive ran out of PermGen space in spark-shell, when those things added up with the Scala compiler. Note that users can still override it by setting their own Java options with this change. Their options will come later in the command string than the -XX:MaxPermSize=128m. Author: Matei Zaharia Closes #823 from mateiz/spark-1879 and squashes the following commits: 6bc0ee8 [Matei Zaharia] Increase MaxPermSize to 128m since some of our builds have lots of classes --- bin/spark-class | 4 ++-- bin/spark-class2.cmd | 4 ++-- .../scala/org/apache/spark/deploy/worker/CommandUtils.scala | 4 +++- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/bin/spark-class b/bin/spark-class index 6480ccb58d6aa..2e57295fd0234 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -99,14 +99,14 @@ else fi # Set JAVA_OPTS to be able to load native libraries and to set heap size -JAVA_OPTS="$OUR_JAVA_OPTS" +JAVA_OPTS="-XX:MaxPermSize=128m $OUR_JAVA_OPTS" JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e "$FWDIR/conf/java-opts" ] ; then JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" fi export JAVA_OPTS -# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! +# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! if [ ! -f "$FWDIR/RELEASE" ]; then # Exit if the user hasn't compiled Spark diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index 266edd9fa9835..e420eb409e529 100755 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -77,8 +77,8 @@ rem All drivers use SPARK_JAVA_OPTS + SPARK_DRIVER_MEMORY. The repl also uses SP ) rem Set JAVA_OPTS to be able to load native libraries and to set heap size -set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% -rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! +set JAVA_OPTS=-XX:MaxPermSize=128m %OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% +rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! rem Test whether the user has built Spark if exist "%FWDIR%RELEASE" goto skip_build_test diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index c7f0f244ea2fc..4af5bc3afad6c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -65,6 +65,8 @@ object CommandUtils extends Logging { Seq() } + val permGenOpt = Seq("-XX:MaxPermSize=128m") + // Figure out our classpath with the external compute-classpath script val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" val classPath = Utils.executeAndGetOutput( @@ -73,7 +75,7 @@ object CommandUtils extends Logging { val userClassPath = command.classPathEntries ++ Seq(classPath) Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++ - libraryOpts ++ extraOpts ++ workerLocalOpts ++ memoryOpts + permGenOpt ++ libraryOpts ++ extraOpts ++ workerLocalOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ From 6a2c5c610c259f62cb12d8cfc18bf59cdb334bb2 Mon Sep 17 00:00:00 2001 From: witgo Date: Mon, 19 May 2014 19:40:29 -0700 Subject: [PATCH 395/641] [SPARK-1875]NoClassDefFoundError: StringUtils when building with hadoop 1.x and hive Author: witgo Closes #824 from witgo/SPARK-1875_commons-lang-2.6 and squashes the following commits: ef7231d [witgo] review commit ead3c3b [witgo] SPARK-1875:NoClassDefFoundError: StringUtils when building against Hadoop 1 --- project/SparkBuild.scala | 3 +-- sql/hive/pom.xml | 8 -------- 2 files changed, 1 insertion(+), 10 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 29dcd8678b476..b9d92340ff75b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -318,7 +318,6 @@ object SparkBuild extends Build { val excludeFastutil = ExclusionRule(organization = "it.unimi.dsi") val excludeJruby = ExclusionRule(organization = "org.jruby") val excludeThrift = ExclusionRule(organization = "org.apache.thrift") - val excludeCommonsLang = ExclusionRule(organization = "commons-lang") val excludeServletApi = ExclusionRule(organization = "javax.servlet", artifact = "servlet-api") def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark", @@ -493,7 +492,7 @@ object SparkBuild extends Build { javaOptions += "-XX:MaxPermSize=1g", libraryDependencies ++= Seq( "org.spark-project.hive" % "hive-metastore" % hiveVersion, - "org.spark-project.hive" % "hive-exec" % hiveVersion excludeAll(excludeCommonsLang, excludeCommonsLogging), + "org.spark-project.hive" % "hive-exec" % hiveVersion excludeAll(excludeCommonsLogging), "org.spark-project.hive" % "hive-serde" % hiveVersion ), // Multiple queries rely on the TestHive singleton. See comments there for more details. diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 4d0b2fa1452a2..9254b70e64a08 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -57,10 +57,6 @@ commons-logging commons-logging - - commons-lang - commons-lang - @@ -80,10 +76,6 @@ commons-logging commons-logging-api - - commons-lang - commons-lang - From b0ce22e071da4cc62ec5e29abf7b1299b8e4a6b0 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 19 May 2014 20:55:26 -0700 Subject: [PATCH 396/641] SPARK-1689: Spark application should die when removed by Master scheduler.error() will mask the error if there are active tasks. Being removed is a cataclysmic event for Spark applications, and should probably be treated as such. Author: Aaron Davidson Closes #832 from aarondav/i-love-u and squashes the following commits: 9f1200f [Aaron Davidson] SPARK-1689: Spark application should die when removed by Master --- .../spark/scheduler/cluster/SparkDeploySchedulerBackend.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 9768670855f80..9c07b3f7b695a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -88,6 +88,8 @@ private[spark] class SparkDeploySchedulerBackend( if (!stopping) { logError("Application has been killed. Reason: " + reason) scheduler.error(reason) + // Ensure the application terminates, as we can no longer run jobs. + sc.stop() } } From bcb9dce6f444a977c714117811bce0c54b417650 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 19 May 2014 21:29:33 -0700 Subject: [PATCH 397/641] [SPARK-1874][MLLIB] Clean up MLlib sample data 1. Added synthetic datasets for `MovieLensALS`, `LinearRegression`, `BinaryClassification`. 2. Embedded instructions in the help message of those example apps. Per discussion with Matei on the JIRA page, new example data is under `data/mllib`. Author: Xiangrui Meng Closes #833 from mengxr/mllib-sample-data and squashes the following commits: 59f0a18 [Xiangrui Meng] add sample binary classification data 3c2f92f [Xiangrui Meng] add linear regression data 050f1ca [Xiangrui Meng] add a sample dataset for MovieLensALS example --- .../sample_binary_classification_data.txt | 100 ++ data/mllib/sample_linear_regression_data.txt | 501 ++++++ data/mllib/sample_movielens_data.txt | 1501 +++++++++++++++++ .../examples/mllib/BinaryClassification.scala | 12 +- .../examples/mllib/LinearRegression.scala | 11 +- .../spark/examples/mllib/MovieLensALS.scala | 15 + 6 files changed, 2138 insertions(+), 2 deletions(-) create mode 100644 data/mllib/sample_binary_classification_data.txt create mode 100755 data/mllib/sample_linear_regression_data.txt create mode 100644 data/mllib/sample_movielens_data.txt diff --git a/data/mllib/sample_binary_classification_data.txt b/data/mllib/sample_binary_classification_data.txt new file mode 100644 index 0000000000000..861c70cde7fd2 --- /dev/null +++ b/data/mllib/sample_binary_classification_data.txt @@ -0,0 +1,100 @@ +0 128:51 129:159 130:253 131:159 132:50 155:48 156:238 157:252 158:252 159:252 160:237 182:54 183:227 184:253 185:252 186:239 187:233 188:252 189:57 190:6 208:10 209:60 210:224 211:252 212:253 213:252 214:202 215:84 216:252 217:253 218:122 236:163 237:252 238:252 239:252 240:253 241:252 242:252 243:96 244:189 245:253 246:167 263:51 264:238 265:253 266:253 267:190 268:114 269:253 270:228 271:47 272:79 273:255 274:168 290:48 291:238 292:252 293:252 294:179 295:12 296:75 297:121 298:21 301:253 302:243 303:50 317:38 318:165 319:253 320:233 321:208 322:84 329:253 330:252 331:165 344:7 345:178 346:252 347:240 348:71 349:19 350:28 357:253 358:252 359:195 372:57 373:252 374:252 375:63 385:253 386:252 387:195 400:198 401:253 402:190 413:255 414:253 415:196 427:76 428:246 429:252 430:112 441:253 442:252 443:148 455:85 456:252 457:230 458:25 467:7 468:135 469:253 470:186 471:12 483:85 484:252 485:223 494:7 495:131 496:252 497:225 498:71 511:85 512:252 513:145 521:48 522:165 523:252 524:173 539:86 540:253 541:225 548:114 549:238 550:253 551:162 567:85 568:252 569:249 570:146 571:48 572:29 573:85 574:178 575:225 576:253 577:223 578:167 579:56 595:85 596:252 597:252 598:252 599:229 600:215 601:252 602:252 603:252 604:196 605:130 623:28 624:199 625:252 626:252 627:253 628:252 629:252 630:233 631:145 652:25 653:128 654:252 655:253 656:252 657:141 658:37 +1 159:124 160:253 161:255 162:63 186:96 187:244 188:251 189:253 190:62 214:127 215:251 216:251 217:253 218:62 241:68 242:236 243:251 244:211 245:31 246:8 268:60 269:228 270:251 271:251 272:94 296:155 297:253 298:253 299:189 323:20 324:253 325:251 326:235 327:66 350:32 351:205 352:253 353:251 354:126 378:104 379:251 380:253 381:184 382:15 405:80 406:240 407:251 408:193 409:23 432:32 433:253 434:253 435:253 436:159 460:151 461:251 462:251 463:251 464:39 487:48 488:221 489:251 490:251 491:172 515:234 516:251 517:251 518:196 519:12 543:253 544:251 545:251 546:89 570:159 571:255 572:253 573:253 574:31 597:48 598:228 599:253 600:247 601:140 602:8 625:64 626:251 627:253 628:220 653:64 654:251 655:253 656:220 681:24 682:193 683:253 684:220 +1 125:145 126:255 127:211 128:31 152:32 153:237 154:253 155:252 156:71 180:11 181:175 182:253 183:252 184:71 209:144 210:253 211:252 212:71 236:16 237:191 238:253 239:252 240:71 264:26 265:221 266:253 267:252 268:124 269:31 293:125 294:253 295:252 296:252 297:108 322:253 323:252 324:252 325:108 350:255 351:253 352:253 353:108 378:253 379:252 380:252 381:108 406:253 407:252 408:252 409:108 434:253 435:252 436:252 437:108 462:255 463:253 464:253 465:170 490:253 491:252 492:252 493:252 494:42 518:149 519:252 520:252 521:252 522:144 546:109 547:252 548:252 549:252 550:144 575:218 576:253 577:253 578:255 579:35 603:175 604:252 605:252 606:253 607:35 631:73 632:252 633:252 634:253 635:35 659:31 660:211 661:252 662:253 663:35 +1 153:5 154:63 155:197 181:20 182:254 183:230 184:24 209:20 210:254 211:254 212:48 237:20 238:254 239:255 240:48 265:20 266:254 267:254 268:57 293:20 294:254 295:254 296:108 321:16 322:239 323:254 324:143 350:178 351:254 352:143 378:178 379:254 380:143 406:178 407:254 408:162 434:178 435:254 436:240 462:113 463:254 464:240 490:83 491:254 492:245 493:31 518:79 519:254 520:246 521:38 547:214 548:254 549:150 575:144 576:241 577:8 603:144 604:240 605:2 631:144 632:254 633:82 659:230 660:247 661:40 687:168 688:209 689:31 +1 152:1 153:168 154:242 155:28 180:10 181:228 182:254 183:100 209:190 210:254 211:122 237:83 238:254 239:162 265:29 266:254 267:248 268:25 293:29 294:255 295:254 296:103 321:29 322:254 323:254 324:109 349:29 350:254 351:254 352:109 377:29 378:254 379:254 380:109 405:29 406:255 407:254 408:109 433:29 434:254 435:254 436:109 461:29 462:254 463:254 464:63 489:29 490:254 491:254 492:28 517:29 518:254 519:254 520:28 545:29 546:254 547:254 548:35 573:29 574:254 575:254 576:109 601:6 602:212 603:254 604:109 630:203 631:254 632:178 658:155 659:254 660:190 686:32 687:199 688:104 +0 130:64 131:253 132:255 133:63 157:96 158:205 159:251 160:253 161:205 162:111 163:4 184:96 185:189 186:251 187:251 188:253 189:251 190:251 191:31 209:16 210:64 211:223 212:244 213:251 214:251 215:211 216:213 217:251 218:251 219:31 236:80 237:181 238:251 239:253 240:251 241:251 242:251 243:94 244:96 245:251 246:251 247:31 263:92 264:253 265:253 266:253 267:255 268:253 269:253 270:253 271:95 272:96 273:253 274:253 275:31 290:92 291:236 292:251 293:243 294:220 295:233 296:251 297:251 298:243 299:82 300:96 301:251 302:251 303:31 317:80 318:253 319:251 320:251 321:188 323:96 324:251 325:251 326:109 328:96 329:251 330:251 331:31 344:96 345:240 346:253 347:243 348:188 349:42 351:96 352:204 353:109 354:4 356:12 357:197 358:251 359:31 372:221 373:251 374:253 375:121 379:36 380:23 385:190 386:251 387:31 399:48 400:234 401:253 413:191 414:253 415:31 426:44 427:221 428:251 429:251 440:12 441:197 442:251 443:31 454:190 455:251 456:251 457:251 468:96 469:251 470:251 471:31 482:190 483:251 484:251 485:113 495:40 496:234 497:251 498:219 499:23 510:190 511:251 512:251 513:94 522:40 523:217 524:253 525:231 526:47 538:191 539:253 540:253 541:253 548:12 549:174 550:253 551:253 552:219 553:39 566:67 567:236 568:251 569:251 570:191 571:190 572:111 573:72 574:190 575:191 576:197 577:251 578:243 579:121 580:39 595:63 596:236 597:251 598:253 599:251 600:251 601:251 602:251 603:253 604:251 605:188 606:94 624:27 625:129 626:253 627:251 628:251 629:251 630:251 631:229 632:168 633:15 654:95 655:212 656:251 657:211 658:94 659:59 +1 159:121 160:254 161:136 186:13 187:230 188:253 189:248 190:99 213:4 214:118 215:253 216:253 217:225 218:42 241:61 242:253 243:253 244:253 245:74 268:32 269:206 270:253 271:253 272:186 273:9 296:211 297:253 298:253 299:239 300:69 324:254 325:253 326:253 327:133 351:142 352:255 353:253 354:186 355:8 378:149 379:229 380:254 381:207 382:21 405:54 406:229 407:253 408:254 409:105 433:152 434:254 435:254 436:213 437:26 460:112 461:251 462:253 463:253 464:26 487:29 488:212 489:253 490:250 491:149 514:36 515:214 516:253 517:253 518:137 542:75 543:253 544:253 545:253 546:59 570:93 571:253 572:253 573:189 574:17 598:224 599:253 600:253 601:84 625:43 626:235 627:253 628:126 629:1 653:99 654:248 655:253 656:119 682:225 683:235 684:49 +1 100:166 101:222 102:55 128:197 129:254 130:218 131:5 155:29 156:249 157:254 158:254 159:9 183:45 184:254 185:254 186:174 187:2 210:4 211:164 212:254 213:254 214:85 238:146 239:254 240:254 241:254 242:85 265:101 266:245 267:254 268:254 269:254 270:85 292:97 293:248 294:254 295:204 296:254 297:254 298:85 315:12 316:59 317:98 318:151 319:237 320:254 321:254 322:109 323:35 324:254 325:254 326:85 343:41 344:216 345:254 346:254 347:239 348:153 349:37 350:4 351:32 352:254 353:254 354:85 372:7 373:44 374:44 375:30 379:32 380:254 381:254 382:96 407:19 408:230 409:254 410:174 436:197 437:254 438:110 464:197 465:254 466:85 492:197 493:253 494:63 515:37 516:54 517:54 518:45 519:26 520:84 521:221 522:84 523:21 524:31 525:162 526:78 540:6 541:41 542:141 543:244 544:254 545:254 546:248 547:236 548:254 549:254 550:254 551:233 552:239 553:254 554:138 567:23 568:167 569:254 570:254 571:254 572:254 573:229 574:228 575:185 576:138 577:138 578:138 579:138 580:138 581:138 582:44 595:113 596:254 597:254 598:254 599:179 600:64 601:5 623:32 624:209 625:183 626:97 +0 155:53 156:255 157:253 158:253 159:253 160:124 183:180 184:253 185:251 186:251 187:251 188:251 189:145 190:62 209:32 210:217 211:241 212:253 213:251 214:251 215:251 216:251 217:253 218:107 237:37 238:251 239:251 240:253 241:251 242:251 243:251 244:251 245:253 246:107 265:166 266:251 267:251 268:253 269:251 270:96 271:148 272:251 273:253 274:107 291:73 292:253 293:253 294:253 295:253 296:130 299:110 300:253 301:255 302:108 319:73 320:251 321:251 322:251 323:251 327:109 328:251 329:253 330:107 347:202 348:251 349:251 350:251 351:225 354:6 355:129 356:251 357:253 358:107 375:150 376:251 377:251 378:251 379:71 382:115 383:251 384:251 385:253 386:107 403:253 404:251 405:251 406:173 407:20 410:217 411:251 412:251 413:253 414:107 430:182 431:255 432:253 433:216 438:218 439:253 440:253 441:182 457:63 458:221 459:253 460:251 461:215 465:84 466:236 467:251 468:251 469:77 485:109 486:251 487:253 488:251 489:215 492:11 493:160 494:251 495:251 496:96 513:109 514:251 515:253 516:251 517:137 520:150 521:251 522:251 523:251 524:71 541:109 542:251 543:253 544:251 545:35 547:130 548:253 549:251 550:251 551:173 552:20 569:110 570:253 571:255 572:253 573:98 574:150 575:253 576:255 577:253 578:164 597:109 598:251 599:253 600:251 601:251 602:251 603:251 604:253 605:251 606:35 625:93 626:241 627:253 628:251 629:251 630:251 631:251 632:216 633:112 634:5 654:103 655:253 656:251 657:251 658:251 659:251 683:124 684:251 685:225 686:71 687:71 +0 128:73 129:253 130:227 131:73 132:21 156:73 157:251 158:251 159:251 160:174 182:16 183:166 184:228 185:251 186:251 187:251 188:122 210:62 211:220 212:253 213:251 214:251 215:251 216:251 217:79 238:79 239:231 240:253 241:251 242:251 243:251 244:251 245:232 246:77 264:145 265:253 266:253 267:253 268:255 269:253 270:253 271:253 272:253 273:255 274:108 292:144 293:251 294:251 295:251 296:253 297:168 298:107 299:169 300:251 301:253 302:189 303:20 318:27 319:89 320:236 321:251 322:235 323:215 324:164 325:15 326:6 327:129 328:251 329:253 330:251 331:35 345:47 346:211 347:253 348:251 349:251 350:142 354:37 355:251 356:251 357:253 358:251 359:35 373:109 374:251 375:253 376:251 377:251 378:142 382:11 383:148 384:251 385:253 386:251 387:164 400:11 401:150 402:253 403:255 404:211 405:25 410:11 411:150 412:253 413:255 414:211 415:25 428:140 429:251 430:251 431:253 432:107 438:37 439:251 440:251 441:211 442:46 456:190 457:251 458:251 459:253 460:128 461:5 466:37 467:251 468:251 469:51 484:115 485:251 486:251 487:253 488:188 489:20 492:32 493:109 494:129 495:251 496:173 497:103 512:217 513:251 514:251 515:201 516:30 520:73 521:251 522:251 523:251 524:71 540:166 541:253 542:253 543:255 544:149 545:73 546:150 547:253 548:255 549:253 550:253 551:143 568:140 569:251 570:251 571:253 572:251 573:251 574:251 575:251 576:253 577:251 578:230 579:61 596:190 597:251 598:251 599:253 600:251 601:251 602:251 603:251 604:242 605:215 606:55 624:21 625:189 626:251 627:253 628:251 629:251 630:251 631:173 632:103 653:31 654:200 655:253 656:251 657:96 658:71 659:20 +1 155:178 156:255 157:105 182:6 183:188 184:253 185:216 186:14 210:14 211:202 212:253 213:253 214:23 238:12 239:199 240:253 241:128 242:6 266:42 267:253 268:253 269:158 294:42 295:253 296:253 297:158 322:155 323:253 324:253 325:158 350:160 351:253 352:253 353:147 378:160 379:253 380:253 381:41 405:17 406:225 407:253 408:235 409:31 433:24 434:253 435:253 436:176 461:24 462:253 463:253 464:176 489:24 490:253 491:253 492:176 517:24 518:253 519:253 520:176 545:24 546:253 547:253 548:162 573:46 574:253 575:253 576:59 601:142 602:253 603:253 604:59 629:142 630:253 631:253 632:59 657:142 658:253 659:202 660:8 685:87 686:253 687:139 +0 154:46 155:105 156:254 157:254 158:254 159:254 160:255 161:239 162:41 180:37 181:118 182:222 183:254 184:253 185:253 186:253 187:253 188:253 189:253 190:211 191:54 207:14 208:200 209:253 210:253 211:254 212:253 213:253 214:253 215:253 216:253 217:253 218:253 219:116 233:16 234:160 235:236 236:253 237:253 238:253 239:254 240:253 241:253 242:246 243:229 244:253 245:253 246:253 247:116 261:99 262:253 263:253 264:253 265:253 266:253 267:254 268:253 269:253 270:213 271:99 272:253 273:253 274:253 275:116 288:25 289:194 290:253 291:253 292:253 293:253 294:131 295:97 296:169 297:253 298:93 299:99 300:253 301:253 302:253 303:116 316:206 317:253 318:253 319:251 320:233 321:127 322:9 324:18 325:38 326:3 327:15 328:171 329:253 330:253 331:116 343:55 344:240 345:253 346:253 347:233 355:31 356:186 357:253 358:253 359:116 371:176 372:253 373:253 374:253 375:127 383:99 384:253 385:253 386:253 387:116 399:176 400:253 401:253 402:131 403:9 411:99 412:253 413:253 414:253 415:116 426:119 427:254 428:254 429:232 430:75 440:158 441:254 442:254 443:117 454:118 455:253 456:253 457:154 468:156 469:253 470:253 471:116 482:118 483:253 484:253 485:154 496:156 497:253 498:253 499:116 509:46 510:222 511:253 512:253 513:154 522:7 523:116 524:246 525:253 526:180 527:9 538:118 539:253 540:253 541:154 550:116 551:253 552:253 553:253 554:174 566:118 567:253 568:253 569:154 577:110 578:246 579:253 580:253 581:240 582:67 594:118 595:253 596:253 597:238 598:215 599:49 600:20 601:20 602:20 603:66 604:215 605:241 606:253 607:245 608:233 609:64 622:82 623:229 624:253 625:253 626:253 627:253 628:253 629:253 630:253 631:254 632:253 633:253 634:240 635:107 651:176 652:253 653:253 654:253 655:253 656:253 657:253 658:253 659:254 660:253 661:253 662:108 679:40 680:239 681:253 682:253 683:253 684:253 685:253 686:253 687:254 688:161 689:57 690:4 +0 152:56 153:105 154:220 155:254 156:63 178:18 179:166 180:233 181:253 182:253 183:253 184:236 185:209 186:209 187:209 188:77 189:18 206:84 207:253 208:253 209:253 210:253 211:253 212:254 213:253 214:253 215:253 216:253 217:172 218:8 233:57 234:238 235:253 236:253 237:253 238:253 239:253 240:254 241:253 242:253 243:253 244:253 245:253 246:119 260:14 261:238 262:253 263:253 264:253 265:253 266:253 267:253 268:179 269:196 270:253 271:253 272:253 273:253 274:238 275:12 288:33 289:253 290:253 291:253 292:253 293:253 294:248 295:134 297:18 298:83 299:237 300:253 301:253 302:253 303:14 316:164 317:253 318:253 319:253 320:253 321:253 322:128 327:57 328:119 329:214 330:253 331:94 343:57 344:248 345:253 346:253 347:253 348:126 349:14 350:4 357:179 358:253 359:248 360:56 371:175 372:253 373:253 374:240 375:190 376:28 385:179 386:253 387:253 388:173 399:209 400:253 401:253 402:178 413:92 414:253 415:253 416:208 427:211 428:254 429:254 430:179 442:135 443:255 444:209 455:209 456:253 457:253 458:90 470:134 471:253 472:208 483:209 484:253 485:253 486:178 497:2 498:142 499:253 500:208 511:209 512:253 513:253 514:214 515:35 525:30 526:253 527:253 528:208 539:165 540:253 541:253 542:253 543:215 544:36 553:163 554:253 555:253 556:164 567:18 568:172 569:253 570:253 571:253 572:214 573:127 574:7 580:72 581:232 582:253 583:171 584:17 596:8 597:182 598:253 599:253 600:253 601:253 602:162 603:56 607:64 608:240 609:253 610:253 611:14 625:7 626:173 627:253 628:253 629:253 630:253 631:245 632:241 633:239 634:239 635:246 636:253 637:225 638:14 639:1 654:18 655:59 656:138 657:224 658:253 659:253 660:254 661:253 662:253 663:253 664:240 665:96 685:37 686:104 687:192 688:255 689:253 690:253 691:182 692:73 +1 130:7 131:176 132:254 133:224 158:51 159:253 160:253 161:223 185:4 186:170 187:253 188:253 189:214 213:131 214:253 215:253 216:217 217:39 241:209 242:253 243:253 244:134 268:75 269:240 270:253 271:239 272:26 296:184 297:253 298:245 299:63 323:142 324:255 325:253 326:185 350:62 351:229 352:254 353:242 354:73 377:54 378:229 379:253 380:254 381:105 405:152 406:254 407:254 408:213 409:26 432:32 433:243 434:253 435:253 436:115 459:2 460:142 461:253 462:253 463:155 487:30 488:253 489:253 490:232 491:55 515:75 516:253 517:253 518:164 542:72 543:232 544:253 545:189 546:17 570:224 571:253 572:253 573:163 597:43 598:235 599:253 600:253 601:195 602:21 625:28 626:231 627:253 628:253 629:184 630:14 654:225 655:253 656:253 657:75 +0 155:21 156:176 157:253 158:253 159:124 182:105 183:176 184:251 185:251 186:251 187:251 188:105 208:58 209:217 210:241 211:253 212:251 213:251 214:251 215:251 216:243 217:113 218:5 235:63 236:231 237:251 238:251 239:253 240:251 241:251 242:251 243:251 244:253 245:251 246:113 263:144 264:251 265:251 266:251 267:253 268:251 269:251 270:251 271:251 272:253 273:251 274:215 290:125 291:253 292:253 293:253 294:253 295:255 296:253 297:253 298:253 299:253 300:255 301:253 302:227 303:42 318:253 319:251 320:251 321:251 322:251 323:253 324:251 325:251 326:251 327:251 328:253 329:251 330:251 331:142 345:27 346:253 347:251 348:251 349:235 350:241 351:253 352:251 353:246 354:137 355:35 356:98 357:251 358:251 359:236 360:61 372:47 373:211 374:253 375:251 376:235 377:82 378:103 379:253 380:251 381:137 384:73 385:251 386:251 387:251 388:71 399:27 400:211 401:251 402:253 403:251 404:86 407:72 408:71 409:10 412:73 413:251 414:251 415:173 416:20 427:89 428:253 429:253 430:255 431:253 432:35 440:73 441:253 442:253 443:253 444:72 454:84 455:236 456:251 457:251 458:253 459:251 460:138 468:73 469:251 470:251 471:251 472:71 481:63 482:236 483:251 484:251 485:251 486:227 487:251 488:246 489:138 490:11 494:16 495:37 496:228 497:251 498:246 499:137 500:10 509:73 510:251 511:251 512:251 513:173 514:42 515:142 516:142 517:142 518:41 522:109 523:251 524:253 525:251 526:137 537:73 538:251 539:251 540:173 541:20 549:27 550:211 551:251 552:253 553:147 554:10 565:73 566:253 567:253 568:143 575:21 576:176 577:253 578:253 579:253 593:73 594:251 595:251 596:205 597:144 603:176 604:251 605:251 606:188 607:107 621:62 622:236 623:251 624:251 625:251 626:218 627:217 628:217 629:217 630:217 631:253 632:230 633:189 634:20 650:83 651:158 652:251 653:251 654:253 655:251 656:251 657:251 658:251 659:253 660:107 679:37 680:251 681:251 682:253 683:251 684:251 685:251 686:122 687:72 688:30 +1 151:68 152:45 153:131 154:131 155:131 156:101 157:68 158:92 159:44 187:19 188:170 211:29 212:112 213:89 215:40 216:222 239:120 240:254 241:251 242:127 243:40 244:222 267:197 268:254 269:254 270:91 271:40 272:222 294:64 295:247 296:254 297:236 298:50 299:40 300:107 322:184 323:254 324:254 325:91 327:6 328:14 350:203 351:254 352:254 353:71 377:23 378:218 379:254 380:254 381:71 405:113 406:254 407:255 408:239 409:53 433:210 434:254 435:254 436:195 460:62 461:242 462:254 463:241 464:88 468:28 488:86 489:254 490:254 491:189 495:28 496:104 516:106 517:254 518:254 519:168 523:40 524:91 544:216 545:254 546:245 547:51 551:35 552:80 572:216 573:254 574:102 599:55 600:239 601:254 602:52 627:166 628:254 629:210 630:23 655:223 656:252 657:104 683:223 684:169 +0 125:29 126:170 127:255 128:255 129:141 151:29 152:198 153:255 154:255 155:255 156:226 157:255 158:86 178:141 179:255 180:255 181:170 182:29 184:86 185:255 186:255 187:141 204:29 205:226 206:255 207:198 208:57 213:226 214:255 215:255 216:226 217:114 231:29 232:255 233:255 234:114 241:141 242:170 243:114 244:255 245:255 246:141 259:226 260:255 261:170 269:29 270:57 273:141 274:255 275:226 286:57 287:255 288:170 302:114 303:255 304:198 314:226 315:255 331:170 332:255 333:57 342:255 343:226 360:255 361:170 370:255 371:170 388:114 389:198 398:255 399:226 416:86 417:255 426:198 427:255 444:86 445:255 454:114 455:255 456:57 472:86 473:255 482:29 483:255 484:226 500:141 501:255 511:170 512:255 513:170 528:226 529:198 539:29 540:226 541:255 542:170 555:29 556:255 557:114 568:29 569:226 570:255 571:141 582:57 583:226 584:226 598:141 599:255 600:255 601:170 602:86 607:29 608:86 609:226 610:255 611:226 612:29 627:86 628:198 629:255 630:255 631:255 632:255 633:255 634:255 635:255 636:255 637:255 638:141 639:29 657:29 658:114 659:170 660:170 661:170 662:170 663:170 664:86 +0 153:203 154:254 155:252 156:252 157:252 158:214 159:51 160:20 180:62 181:221 182:252 183:250 184:250 185:250 186:252 187:250 188:160 189:20 207:62 208:211 209:250 210:252 211:250 212:250 213:250 214:252 215:250 216:250 217:49 234:41 235:221 236:250 237:250 238:252 239:250 240:250 241:250 242:252 243:250 244:128 245:10 262:254 263:252 264:252 265:252 266:254 267:252 268:252 269:252 270:254 271:252 272:252 273:90 290:150 291:190 292:250 293:250 294:252 295:250 296:250 297:169 298:171 299:250 300:250 301:250 302:82 318:31 319:191 320:250 321:250 322:252 323:189 324:100 325:20 326:172 327:250 328:250 329:250 330:80 346:213 347:250 348:250 349:250 350:212 351:29 354:252 355:250 356:250 357:250 374:92 375:252 376:252 377:252 382:51 383:252 384:252 385:252 386:203 401:82 402:252 403:250 404:250 405:169 410:132 411:250 412:250 413:250 414:121 428:92 429:231 430:252 431:250 432:159 433:20 438:252 439:250 440:250 441:250 456:30 457:211 458:252 459:250 460:221 461:40 466:90 467:250 468:250 469:250 470:163 484:31 485:213 486:254 487:232 488:80 494:92 495:252 496:252 497:212 498:163 512:151 513:250 514:252 515:149 522:252 523:250 524:250 525:49 540:60 541:221 542:252 543:210 544:60 550:252 551:250 552:250 553:49 569:202 570:252 571:250 572:221 573:40 576:123 577:202 578:252 579:250 580:250 581:49 596:123 597:243 598:255 599:252 600:252 601:252 602:254 603:252 604:252 605:252 606:254 607:252 608:100 625:121 626:171 627:250 628:250 629:250 630:252 631:250 632:250 633:250 634:252 635:250 636:100 654:20 655:160 656:250 657:250 658:252 659:250 660:250 661:250 662:252 663:189 664:40 683:20 684:170 685:250 686:252 687:250 688:128 689:49 690:49 691:29 +1 98:64 99:191 100:70 125:68 126:243 127:253 128:249 129:63 152:30 153:223 154:253 155:253 156:247 157:41 179:73 180:238 181:253 182:253 183:253 184:242 206:73 207:236 208:253 209:253 210:253 211:253 212:242 234:182 235:253 236:253 237:191 238:247 239:253 240:149 262:141 263:253 264:143 265:86 266:249 267:253 268:122 290:9 291:36 292:7 293:14 294:233 295:253 296:122 322:230 323:253 324:122 350:230 351:253 352:122 378:231 379:255 380:123 406:230 407:253 408:52 433:61 434:245 435:253 461:98 462:253 463:253 468:35 469:12 489:98 490:253 491:253 494:9 495:142 496:233 497:146 517:190 518:253 519:253 520:128 521:7 522:99 523:253 524:253 525:180 544:29 545:230 546:253 547:253 548:252 549:210 550:253 551:253 552:253 553:140 571:28 572:207 573:253 574:253 575:253 576:254 577:253 578:253 579:235 580:70 581:9 599:126 600:253 601:253 602:253 603:253 604:254 605:253 606:168 607:19 627:79 628:253 629:253 630:201 631:190 632:132 633:63 634:5 +1 125:26 126:240 127:72 153:25 154:238 155:208 182:209 183:226 184:14 210:209 211:254 212:43 238:175 239:254 240:128 266:63 267:254 268:204 294:107 295:254 296:204 322:88 323:254 324:204 350:55 351:254 352:204 378:126 379:254 380:204 406:126 407:254 408:189 434:169 435:254 436:121 462:209 463:254 464:193 490:209 491:254 492:111 517:22 518:235 519:254 520:37 545:137 546:254 547:227 548:16 573:205 574:255 575:185 601:205 602:254 603:125 629:205 630:254 631:125 657:111 658:212 659:43 +0 155:62 156:91 157:213 158:255 159:228 160:91 161:12 182:70 183:230 184:253 185:253 186:253 187:253 188:253 189:152 190:7 210:246 211:253 212:253 213:253 214:253 215:253 216:253 217:253 218:106 237:21 238:247 239:253 240:253 241:253 242:253 243:253 244:253 245:208 246:24 265:156 266:253 267:253 268:253 269:253 270:253 271:253 272:253 273:195 292:88 293:238 294:253 295:253 296:253 297:221 298:253 299:253 300:253 301:195 320:230 321:253 322:253 323:253 324:198 325:40 326:177 327:253 328:253 329:195 346:56 347:156 348:251 349:253 350:189 351:182 352:15 354:86 355:240 356:253 357:210 358:28 374:213 375:253 376:253 377:156 378:3 383:205 384:253 385:253 386:106 401:121 402:252 403:253 404:135 405:3 411:46 412:253 413:253 414:106 428:28 429:212 430:253 431:248 432:23 439:42 440:253 441:253 442:106 456:197 457:253 458:234 459:70 467:42 468:253 469:253 470:106 483:11 484:202 485:253 486:187 495:58 496:253 497:210 498:27 511:107 512:253 513:253 514:40 522:53 523:227 524:253 525:195 539:107 540:253 541:253 542:40 549:47 550:227 551:253 552:231 553:58 567:107 568:253 569:253 570:40 575:5 576:131 577:222 578:253 579:231 580:59 595:14 596:204 597:253 598:226 599:222 600:73 601:58 602:58 603:170 604:253 605:253 606:227 607:58 624:197 625:253 626:253 627:253 628:253 629:253 630:253 631:253 632:253 633:238 634:58 652:33 653:179 654:241 655:253 656:253 657:253 658:253 659:250 660:116 661:14 682:75 683:179 684:253 685:151 686:89 687:86 +1 157:42 158:228 159:253 160:253 185:144 186:251 187:251 188:251 212:89 213:236 214:251 215:235 216:215 239:79 240:253 241:251 242:251 243:142 267:180 268:253 269:251 270:251 271:142 294:32 295:202 296:255 297:253 298:216 322:109 323:251 324:253 325:251 326:112 349:6 350:129 351:251 352:253 353:127 354:5 377:37 378:251 379:251 380:253 381:107 405:166 406:251 407:251 408:201 409:30 432:42 433:228 434:253 435:253 460:144 461:251 462:251 463:147 487:63 488:236 489:251 490:251 491:71 515:150 516:251 517:251 518:204 519:41 543:253 544:251 545:251 546:142 571:255 572:253 573:164 598:105 599:253 600:251 601:35 626:180 627:253 628:251 629:35 654:180 655:253 656:251 657:35 682:180 683:253 684:251 685:35 +1 128:62 129:254 130:213 156:102 157:253 158:252 159:102 160:20 184:102 185:254 186:253 187:254 188:50 212:102 213:253 214:252 215:253 216:50 240:102 241:254 242:253 243:254 244:50 268:142 269:253 270:252 271:253 272:50 295:51 296:253 297:254 298:253 299:224 300:20 323:132 324:252 325:253 326:252 327:162 351:173 352:253 353:254 354:253 355:102 378:82 379:253 380:252 381:253 382:252 383:61 406:203 407:254 408:253 409:254 410:233 433:41 434:243 435:253 436:252 437:253 438:111 461:132 462:253 463:254 464:253 465:203 488:41 489:253 490:252 491:253 492:252 493:40 515:11 516:213 517:254 518:253 519:254 520:151 543:92 544:252 545:253 546:252 547:192 548:50 570:21 571:214 572:253 573:255 574:253 575:41 598:142 599:253 600:252 601:253 602:171 625:113 626:253 627:255 628:253 629:203 630:40 653:30 654:131 655:233 656:111 +0 154:28 155:195 156:254 157:254 158:254 159:254 160:254 161:255 162:61 181:6 182:191 183:253 184:253 185:253 186:253 187:253 188:253 189:253 190:60 208:26 209:190 210:253 211:253 212:253 213:253 214:240 215:191 216:242 217:253 218:60 235:15 236:187 237:253 238:253 239:253 240:253 241:253 242:200 244:211 245:253 246:60 262:22 263:66 264:253 265:253 266:253 267:253 268:241 269:209 270:44 271:23 272:218 273:253 274:60 290:124 291:253 292:253 293:253 294:253 295:253 296:182 299:131 300:253 301:253 302:60 318:38 319:217 320:253 321:253 322:244 323:111 324:37 327:131 328:253 329:253 330:60 346:124 347:253 348:253 349:253 350:165 354:22 355:182 356:253 357:253 358:60 374:124 375:253 376:253 377:240 378:45 382:53 383:253 384:253 385:249 386:58 401:16 402:168 403:253 404:216 405:45 410:53 411:253 412:253 413:138 429:159 430:253 431:253 432:147 438:53 439:253 440:253 441:138 456:136 457:252 458:253 459:227 460:5 466:53 467:253 468:243 469:101 484:140 485:253 486:253 487:124 494:156 495:253 496:218 511:13 512:164 513:253 514:142 515:5 521:32 522:233 523:253 524:218 539:62 540:253 541:253 542:130 548:37 549:203 550:253 551:253 552:127 567:62 568:253 569:253 570:147 571:36 572:36 573:36 574:36 575:151 576:222 577:253 578:245 579:127 580:8 595:34 596:202 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:253 605:253 606:200 624:140 625:253 626:253 627:253 628:253 629:253 630:253 631:253 632:248 633:235 634:65 652:87 653:173 654:253 655:253 656:253 657:253 658:253 659:253 660:182 681:14 682:78 683:96 684:253 685:253 686:253 687:137 688:56 +0 123:8 124:76 125:202 126:254 127:255 128:163 129:37 130:2 150:13 151:182 152:253 153:253 154:253 155:253 156:253 157:253 158:23 177:15 178:179 179:253 180:253 181:212 182:91 183:218 184:253 185:253 186:179 187:109 205:105 206:253 207:253 208:160 209:35 210:156 211:253 212:253 213:253 214:253 215:250 216:113 232:19 233:212 234:253 235:253 236:88 237:121 238:253 239:233 240:128 241:91 242:245 243:253 244:248 245:114 260:104 261:253 262:253 263:110 264:2 265:142 266:253 267:90 270:26 271:199 272:253 273:248 274:63 287:1 288:173 289:253 290:253 291:29 293:84 294:228 295:39 299:72 300:251 301:253 302:215 303:29 315:36 316:253 317:253 318:203 319:13 328:82 329:253 330:253 331:170 343:36 344:253 345:253 346:164 356:11 357:198 358:253 359:184 360:6 371:36 372:253 373:253 374:82 385:138 386:253 387:253 388:35 399:128 400:253 401:253 402:47 413:48 414:253 415:253 416:35 427:154 428:253 429:253 430:47 441:48 442:253 443:253 444:35 455:102 456:253 457:253 458:99 469:48 470:253 471:253 472:35 483:36 484:253 485:253 486:164 496:16 497:208 498:253 499:211 500:17 511:32 512:244 513:253 514:175 515:4 524:44 525:253 526:253 527:156 540:171 541:253 542:253 543:29 551:30 552:217 553:253 554:188 555:19 568:171 569:253 570:253 571:59 578:60 579:217 580:253 581:253 582:70 596:78 597:253 598:253 599:231 600:48 604:26 605:128 606:249 607:253 608:244 609:94 610:15 624:8 625:151 626:253 627:253 628:234 629:101 630:121 631:219 632:229 633:253 634:253 635:201 636:80 653:38 654:232 655:253 656:253 657:253 658:253 659:253 660:253 661:253 662:201 663:66 +0 127:68 128:254 129:255 130:254 131:107 153:11 154:176 155:230 156:253 157:253 158:253 159:212 180:28 181:197 182:253 183:253 184:253 185:253 186:253 187:229 188:107 189:14 208:194 209:253 210:253 211:253 212:253 213:253 214:253 215:253 216:253 217:53 235:69 236:241 237:253 238:253 239:253 240:253 241:241 242:186 243:253 244:253 245:195 262:10 263:161 264:253 265:253 266:253 267:246 268:40 269:57 270:231 271:253 272:253 273:195 290:140 291:253 292:253 293:253 294:253 295:154 297:25 298:253 299:253 300:253 301:195 318:213 319:253 320:253 321:253 322:135 323:8 325:3 326:128 327:253 328:253 329:195 345:77 346:238 347:253 348:253 349:253 350:7 354:116 355:253 356:253 357:195 372:11 373:165 374:253 375:253 376:231 377:70 378:1 382:78 383:237 384:253 385:195 400:33 401:253 402:253 403:253 404:182 411:200 412:253 413:195 428:98 429:253 430:253 431:253 432:24 439:42 440:253 441:195 456:197 457:253 458:253 459:253 460:24 467:163 468:253 469:195 484:197 485:253 486:253 487:189 488:13 494:53 495:227 496:253 497:121 512:197 513:253 514:253 515:114 521:21 522:227 523:253 524:231 525:27 540:197 541:253 542:253 543:114 547:5 548:131 549:143 550:253 551:231 552:59 568:197 569:253 570:253 571:236 572:73 573:58 574:217 575:223 576:253 577:253 578:253 579:174 596:197 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:253 605:253 606:253 607:48 624:149 625:253 626:253 627:253 628:253 629:253 630:253 631:253 632:253 633:182 634:15 635:3 652:12 653:168 654:253 655:253 656:253 657:253 658:253 659:248 660:89 661:23 +1 157:85 158:255 159:103 160:1 185:205 186:253 187:253 188:30 213:205 214:253 215:253 216:30 240:44 241:233 242:253 243:244 244:27 268:135 269:253 270:253 271:100 296:153 297:253 298:240 299:76 323:12 324:208 325:253 326:166 351:69 352:253 353:253 354:142 378:14 379:110 380:253 381:235 382:33 406:63 407:223 408:235 409:130 434:186 435:253 436:235 437:37 461:17 462:145 463:253 464:231 465:35 489:69 490:220 491:231 492:123 516:18 517:205 518:253 519:176 520:27 543:17 544:125 545:253 546:185 547:39 571:71 572:214 573:231 574:41 599:167 600:253 601:225 602:33 626:72 627:205 628:207 629:14 653:30 654:249 655:233 656:49 681:32 682:253 683:89 +1 126:94 127:132 154:250 155:250 156:4 182:250 183:254 184:95 210:250 211:254 212:95 238:250 239:254 240:95 266:250 267:254 268:95 294:250 295:254 296:95 322:250 323:254 324:95 350:250 351:254 352:95 378:250 379:254 380:95 405:77 406:254 407:250 408:19 433:96 434:254 435:249 461:53 462:253 463:252 464:43 490:250 491:251 492:32 517:85 518:254 519:249 545:96 546:254 547:249 573:83 574:254 575:250 576:14 602:250 603:254 604:95 630:250 631:255 632:95 658:132 659:254 660:95 +1 124:32 125:253 126:31 152:32 153:251 154:149 180:32 181:251 182:188 208:32 209:251 210:188 236:32 237:251 238:228 239:59 264:32 265:253 266:253 267:95 292:28 293:236 294:251 295:114 321:127 322:251 323:251 349:127 350:251 351:251 377:48 378:232 379:251 406:223 407:253 408:159 434:221 435:251 436:158 462:142 463:251 464:158 490:64 491:251 492:242 493:55 518:64 519:251 520:253 521:161 546:64 547:253 548:255 549:221 574:16 575:181 576:253 577:220 603:79 604:253 605:236 606:63 632:213 633:251 634:126 660:96 661:251 662:126 +1 129:39 130:254 131:255 132:254 133:140 157:136 158:253 159:253 160:228 161:67 184:6 185:227 186:253 187:253 188:58 211:29 212:188 213:253 214:253 215:253 216:17 239:95 240:253 241:253 242:253 243:157 244:8 266:3 267:107 268:253 269:253 270:245 271:77 294:29 295:253 296:253 297:240 298:100 322:141 323:253 324:253 325:215 349:129 350:248 351:253 352:253 353:215 377:151 378:253 379:253 380:253 381:144 405:151 406:253 407:253 408:253 409:27 431:3 432:102 433:242 434:253 435:253 436:110 437:3 459:97 460:253 461:253 462:253 463:214 464:55 487:207 488:253 489:253 490:253 491:158 515:67 516:253 517:253 518:253 519:158 543:207 544:253 545:253 546:240 547:88 571:207 572:253 573:253 574:224 598:32 599:217 600:253 601:253 602:224 626:141 627:253 628:253 629:253 630:133 654:36 655:219 656:253 657:140 658:10 +0 123:59 124:55 149:71 150:192 151:254 152:250 153:147 154:17 176:123 177:247 178:253 179:254 180:253 181:253 182:196 183:79 184:176 185:175 186:175 187:124 188:48 203:87 204:247 205:247 206:176 207:95 208:102 209:117 210:243 211:237 212:192 213:232 214:253 215:253 216:245 217:152 218:6 230:23 231:229 232:253 233:138 238:219 239:58 241:95 242:118 243:80 244:230 245:254 246:196 247:30 258:120 259:254 260:205 261:8 266:114 272:38 273:255 274:254 275:155 276:5 286:156 287:253 288:92 301:61 302:235 303:253 304:102 314:224 315:253 316:78 330:117 331:253 332:196 333:18 342:254 343:253 344:78 358:9 359:211 360:253 361:73 370:254 371:253 372:78 387:175 388:253 389:155 398:194 399:254 400:101 415:79 416:254 417:155 426:112 427:253 428:211 429:9 443:73 444:251 445:200 454:41 455:241 456:253 457:87 471:25 472:240 473:253 483:147 484:253 485:227 486:47 499:94 500:253 501:200 511:5 512:193 513:253 514:230 515:76 527:175 528:253 529:155 540:31 541:219 542:254 543:255 544:126 545:18 553:14 554:149 555:254 556:244 557:45 569:21 570:158 571:254 572:253 573:226 574:162 575:118 576:96 577:20 578:20 579:73 580:118 581:224 582:253 583:247 584:85 598:30 599:155 600:253 601:253 602:253 603:253 604:254 605:253 606:253 607:253 608:253 609:254 610:247 611:84 627:5 628:27 629:117 630:206 631:244 632:229 633:213 634:213 635:213 636:176 637:117 638:32 659:45 660:23 +1 128:58 129:139 156:247 157:247 158:25 183:121 184:253 185:156 186:3 211:133 212:253 213:145 238:11 239:227 240:253 241:145 266:7 267:189 268:253 269:145 294:35 295:252 296:253 297:145 322:146 323:252 324:253 325:131 350:146 351:252 352:253 353:13 378:146 379:252 380:253 381:13 406:147 407:253 408:255 409:13 434:146 435:252 436:253 437:13 462:146 463:252 464:253 465:13 490:146 491:252 492:253 493:13 517:22 518:230 519:252 520:221 521:9 545:22 546:230 547:252 548:133 574:146 575:252 576:133 602:146 603:252 604:120 630:146 631:252 658:146 659:252 +1 129:28 130:247 131:255 132:165 156:47 157:221 158:252 159:252 160:164 184:177 185:252 186:252 187:252 188:164 212:177 213:252 214:252 215:223 216:78 240:177 241:252 242:252 243:197 267:114 268:236 269:252 270:235 271:42 294:5 295:148 296:252 297:252 298:230 321:14 322:135 323:252 324:252 325:252 326:230 349:78 350:252 351:252 352:252 353:252 354:162 377:78 378:252 379:252 380:252 381:252 382:9 405:78 406:252 407:252 408:252 409:252 410:9 432:32 433:200 434:252 435:252 436:252 437:105 438:3 459:10 460:218 461:252 462:252 463:252 464:105 465:8 487:225 488:252 489:252 490:252 491:240 492:69 514:44 515:237 516:252 517:252 518:228 519:85 541:59 542:218 543:252 544:252 545:225 546:93 568:65 569:208 570:252 571:252 572:252 573:175 596:133 597:252 598:252 599:252 600:225 601:68 624:133 625:252 626:252 627:244 628:54 652:133 653:252 654:252 655:48 +0 156:13 157:6 181:10 182:77 183:145 184:253 185:190 186:67 207:11 208:77 209:193 210:252 211:252 212:253 213:252 214:238 215:157 216:71 217:26 233:10 234:78 235:193 236:252 237:252 238:252 239:252 240:253 241:252 242:252 243:252 244:252 245:228 246:128 247:49 248:5 259:6 260:78 261:194 262:252 263:252 264:252 265:252 266:252 267:252 268:253 269:217 270:192 271:232 272:252 273:252 274:252 275:252 276:135 277:3 286:4 287:147 288:252 289:252 290:252 291:252 292:252 293:252 294:252 295:252 296:175 297:26 299:40 300:145 301:235 302:252 303:252 304:252 305:104 314:208 315:252 316:252 317:252 318:252 319:252 320:252 321:133 322:48 323:48 329:71 330:236 331:252 332:252 333:230 342:253 343:185 344:170 345:252 346:252 347:252 348:173 349:22 358:102 359:252 360:252 361:252 370:24 371:141 372:243 373:252 374:252 375:186 376:5 386:8 387:220 388:252 389:252 398:70 399:247 400:252 401:252 402:165 403:37 414:81 415:251 416:252 417:194 426:255 427:253 428:253 429:251 430:69 441:39 442:231 443:253 444:253 445:127 454:253 455:252 456:249 457:127 468:6 469:147 470:252 471:252 472:190 473:5 482:253 483:252 484:216 495:7 496:145 497:252 498:252 499:252 500:69 510:253 511:252 512:223 513:16 522:25 523:185 524:252 525:252 526:252 527:107 528:8 538:167 539:252 540:252 541:181 542:18 549:105 550:191 551:252 552:252 553:235 554:151 555:10 566:37 567:221 568:252 569:252 570:210 571:193 572:96 573:73 574:130 575:188 576:194 577:227 578:252 579:252 580:235 581:128 595:97 596:220 597:252 598:252 599:252 600:252 601:252 602:252 603:252 604:253 605:252 606:252 607:236 608:70 624:40 625:174 626:252 627:252 628:252 629:252 630:252 631:252 632:253 633:197 634:138 635:29 653:5 654:23 655:116 656:143 657:143 658:143 659:143 660:24 661:10 +0 127:28 128:164 129:254 130:233 131:148 132:11 154:3 155:164 156:254 157:234 158:225 159:254 160:204 182:91 183:254 184:235 185:48 186:32 187:166 188:251 189:92 208:33 209:111 210:214 211:205 212:49 215:24 216:216 217:210 235:34 236:217 237:254 238:254 239:211 244:87 245:237 246:43 262:34 263:216 264:254 265:254 266:252 267:243 268:61 272:38 273:248 274:182 290:171 291:254 292:184 293:205 294:175 295:36 301:171 302:227 317:28 318:234 319:190 320:13 321:193 322:157 329:124 330:238 331:26 345:140 346:254 347:131 349:129 350:157 357:124 358:254 359:95 373:201 374:238 375:56 377:70 378:103 385:124 386:254 387:148 400:62 401:255 402:210 413:150 414:254 415:122 428:86 429:254 430:201 431:15 440:28 441:237 442:246 443:44 456:128 457:254 458:143 468:34 469:243 470:227 484:62 485:254 486:210 496:58 497:249 498:179 512:30 513:240 514:210 524:207 525:254 526:64 541:216 542:231 543:34 551:129 552:248 553:170 554:9 569:131 570:254 571:170 577:17 578:129 579:248 580:225 581:24 597:50 598:245 599:245 600:184 601:106 602:106 603:106 604:133 605:231 606:254 607:244 608:53 626:67 627:249 628:254 629:254 630:254 631:254 632:254 633:251 634:193 635:40 655:38 656:157 657:248 658:166 659:166 660:139 661:57 +0 129:105 130:255 131:219 132:67 133:67 134:52 156:20 157:181 158:253 159:253 160:253 161:253 162:226 163:69 182:4 183:129 184:206 185:253 186:253 187:253 188:253 189:253 190:253 191:130 209:9 210:141 211:253 212:253 213:253 214:253 215:253 216:253 217:253 218:253 219:166 220:20 237:134 238:253 239:253 240:253 241:253 242:253 243:253 244:253 245:253 246:253 247:253 248:65 262:2 263:83 264:207 265:246 266:253 267:253 268:253 269:253 270:253 271:249 272:234 273:247 274:253 275:253 276:65 290:83 291:253 292:253 293:253 294:253 295:253 296:189 297:253 298:253 299:205 301:179 302:253 303:253 304:65 317:85 318:234 319:253 320:253 321:253 322:253 323:157 324:26 325:164 326:151 327:83 329:179 330:253 331:253 332:65 344:65 345:237 346:253 347:253 348:253 349:67 350:36 351:14 353:15 354:12 357:179 358:253 359:253 360:65 371:4 372:141 373:253 374:253 375:221 376:158 377:23 385:179 386:253 387:253 388:65 399:129 400:253 401:253 402:241 403:62 412:72 413:226 414:253 415:175 416:24 426:119 427:247 428:253 429:253 430:206 439:8 440:134 441:253 442:253 443:130 454:132 455:253 456:253 457:194 458:27 467:125 468:253 469:253 470:253 471:130 481:45 482:213 483:253 484:253 485:112 493:70 494:170 495:247 496:253 497:253 498:89 499:43 509:67 510:253 511:253 512:196 513:55 514:9 520:8 521:131 522:253 523:253 524:253 525:86 526:1 537:67 538:253 539:253 540:253 541:253 542:129 546:43 547:114 548:134 549:253 550:253 551:231 552:139 553:41 565:20 566:167 567:253 568:253 569:253 570:247 571:179 572:179 573:179 574:206 575:253 576:253 577:253 578:253 579:72 594:103 595:240 596:253 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:244 605:119 606:8 607:1 623:107 624:253 625:253 626:253 627:253 628:253 629:253 630:253 631:175 632:111 651:3 652:121 653:253 654:253 655:253 656:253 657:253 658:182 659:24 +0 125:22 126:183 127:252 128:254 129:252 130:252 131:252 132:76 151:85 152:85 153:168 154:250 155:250 156:252 157:250 158:250 159:250 160:250 161:71 163:43 164:85 165:14 178:107 179:252 180:250 181:250 182:250 183:250 184:252 185:250 186:250 187:250 188:250 189:210 191:127 192:250 193:146 205:114 206:237 207:252 208:250 209:250 210:250 211:250 212:252 213:250 214:250 215:250 216:250 217:210 219:127 220:250 221:250 232:107 233:237 234:250 235:252 236:250 237:250 238:250 239:74 240:41 241:41 242:41 243:41 244:217 245:34 247:127 248:250 249:250 259:15 260:148 261:252 262:252 263:254 264:238 265:105 275:128 276:252 277:252 286:15 287:140 288:250 289:250 290:250 291:167 292:111 303:127 304:250 305:250 314:43 315:250 316:250 317:250 318:250 331:127 332:250 333:250 342:183 343:250 344:250 345:250 346:110 358:57 359:210 360:250 361:250 370:252 371:250 372:250 373:110 374:7 386:85 387:250 388:250 389:250 398:254 399:252 400:252 401:83 414:86 415:252 416:252 417:217 426:252 427:250 428:250 429:138 430:14 441:15 442:140 443:250 444:250 445:41 454:252 455:250 456:250 457:250 458:41 469:43 470:250 471:250 472:250 473:41 482:252 483:250 484:250 485:250 486:181 497:183 498:250 499:250 500:250 501:41 510:76 511:250 512:250 513:250 514:250 524:177 525:252 526:250 527:250 528:110 529:7 538:36 539:224 540:252 541:252 542:252 543:219 544:43 545:43 546:43 547:7 549:15 550:43 551:183 552:252 553:255 554:252 555:126 567:85 568:250 569:250 570:250 571:252 572:250 573:250 574:250 575:111 576:86 577:140 578:250 579:250 580:250 581:252 582:222 583:83 595:42 596:188 597:250 598:250 599:252 600:250 601:250 602:250 603:250 604:252 605:250 606:250 607:250 608:250 609:126 610:83 624:127 625:250 626:250 627:252 628:250 629:250 630:250 631:250 632:252 633:250 634:250 635:137 636:83 652:21 653:41 654:217 655:252 656:250 657:250 658:250 659:250 660:217 661:41 662:41 663:14 +1 155:114 156:206 157:25 183:238 184:252 185:55 211:222 212:252 213:55 239:113 240:252 241:55 267:113 268:252 269:55 295:255 296:253 297:56 323:253 324:176 325:6 350:32 351:253 352:233 353:43 378:140 379:253 380:195 381:19 406:140 407:253 408:167 433:29 434:253 435:141 461:29 462:252 463:140 489:29 490:252 491:140 517:29 518:252 519:140 545:29 546:252 547:140 573:169 574:253 575:79 601:169 602:252 628:76 629:234 630:141 656:197 657:233 658:37 684:197 685:223 +1 127:73 128:253 129:253 130:63 155:115 156:252 157:252 158:144 183:217 184:252 185:252 186:144 210:63 211:237 212:252 213:252 214:144 238:109 239:252 240:252 241:252 266:109 267:252 268:252 269:252 294:109 295:252 296:252 297:252 322:191 323:252 324:252 325:252 349:145 350:255 351:253 352:253 353:253 376:32 377:237 378:253 379:252 380:252 381:210 404:37 405:252 406:253 407:252 408:252 409:108 432:37 433:252 434:253 435:252 436:252 437:108 460:21 461:207 462:255 463:253 464:253 465:108 489:144 490:253 491:252 492:252 493:108 516:27 517:221 518:253 519:252 520:252 521:108 544:16 545:190 546:253 547:252 548:252 549:108 573:145 574:255 575:253 576:253 577:253 601:144 602:253 603:252 604:252 605:210 629:144 630:253 631:252 632:252 633:108 657:62 658:253 659:252 660:252 661:108 +1 120:85 121:253 122:132 123:9 147:82 148:241 149:251 150:251 151:128 175:175 176:251 177:251 178:251 179:245 180:121 203:13 204:204 205:251 206:251 207:251 208:245 209:107 232:39 233:251 234:251 235:251 236:251 237:167 238:22 260:15 261:155 262:251 263:251 264:251 265:251 266:177 289:15 290:157 291:248 292:251 293:251 294:251 295:165 319:214 320:251 321:251 322:251 323:212 324:78 325:24 347:109 348:251 349:251 350:251 351:253 352:251 353:170 354:10 375:5 376:57 377:162 378:251 379:253 380:251 381:251 382:18 405:106 406:239 407:255 408:253 409:253 410:213 434:105 435:253 436:251 437:251 438:230 439:72 463:253 464:251 465:251 466:251 467:221 468:67 491:72 492:251 493:251 494:251 495:251 496:96 519:36 520:199 521:251 522:251 523:251 524:155 525:15 548:45 549:204 550:251 551:251 552:251 553:157 577:161 578:249 579:251 580:251 581:248 582:147 606:233 607:251 608:251 609:251 610:173 634:233 635:251 636:251 637:251 638:173 662:53 663:131 664:251 665:251 666:173 +1 126:15 127:200 128:255 129:90 154:42 155:254 156:254 157:173 182:42 183:254 184:254 185:199 210:26 211:237 212:254 213:221 214:12 239:213 240:254 241:231 242:17 267:213 268:254 269:199 295:213 296:254 297:199 323:213 324:254 325:96 350:20 351:232 352:254 353:33 378:84 379:254 380:229 381:17 406:168 407:254 408:203 433:8 434:217 435:254 436:187 461:84 462:254 463:254 464:48 489:195 490:254 491:254 492:37 516:20 517:233 518:254 519:212 520:4 544:132 545:254 546:254 547:82 571:9 572:215 573:254 574:254 575:116 576:46 599:55 600:254 601:254 602:254 603:254 604:121 627:113 628:254 629:254 630:254 631:254 632:40 655:12 656:163 657:254 658:185 659:58 660:1 +0 182:32 183:57 184:57 185:57 186:57 187:57 188:57 189:57 208:67 209:185 210:229 211:252 212:252 213:252 214:253 215:252 216:252 217:252 218:185 219:66 234:13 235:188 236:246 237:252 238:253 239:252 240:252 241:252 242:241 243:139 244:177 245:252 246:253 247:246 248:187 249:13 261:26 262:255 263:253 264:244 265:175 266:101 274:126 275:244 276:253 277:153 288:82 289:243 290:253 291:214 292:81 303:169 304:252 305:252 315:19 316:215 317:252 318:206 319:56 331:169 332:252 333:252 343:157 344:252 345:252 346:13 359:169 360:252 361:151 370:41 371:253 372:253 373:128 386:92 387:253 388:206 389:13 398:166 399:252 400:196 401:9 414:216 415:252 416:142 426:253 427:252 428:168 441:89 442:253 443:208 444:13 454:253 455:252 456:68 468:38 469:225 470:253 471:96 482:254 483:253 484:56 495:45 496:229 497:253 498:151 510:253 511:252 512:81 522:70 523:225 524:252 525:227 538:216 539:252 540:168 548:29 549:134 550:253 551:252 552:186 553:31 566:91 567:252 568:243 569:125 573:51 574:114 575:113 576:210 577:252 578:253 579:151 580:19 595:157 596:253 597:253 598:254 599:253 600:253 601:253 602:254 603:253 604:244 605:175 606:51 623:19 624:122 625:196 626:197 627:221 628:196 629:196 630:197 631:121 632:56 655:25 +0 127:42 128:235 129:255 130:84 153:15 154:132 155:208 156:253 157:253 158:171 159:108 180:6 181:177 182:253 183:253 184:253 185:253 186:253 187:242 188:110 208:151 209:253 210:253 211:253 212:253 213:253 214:253 215:253 216:139 235:48 236:208 237:253 238:253 239:253 240:253 241:253 242:253 243:253 244:139 263:85 264:253 265:253 266:253 267:253 268:236 269:156 270:184 271:253 272:148 273:6 290:7 291:141 292:253 293:253 294:253 295:253 296:27 298:170 299:253 300:253 301:74 318:19 319:253 320:253 321:253 322:253 323:253 324:27 326:170 327:253 328:253 329:74 345:16 346:186 347:253 348:253 349:253 350:242 351:105 352:4 354:170 355:253 356:253 357:94 358:1 373:141 374:253 375:253 376:253 377:242 378:100 382:170 383:253 384:253 385:253 386:8 401:141 402:253 403:253 404:253 405:224 410:170 411:253 412:253 413:253 414:8 428:12 429:158 430:253 431:253 432:230 433:51 438:18 439:237 440:253 441:253 442:8 456:76 457:253 458:253 459:218 460:61 467:236 468:253 469:253 470:8 484:76 485:253 486:253 487:168 495:110 496:253 497:132 498:3 512:76 513:253 514:253 515:168 521:20 522:174 523:239 524:147 525:5 539:5 540:155 541:253 542:253 543:168 548:102 549:170 550:253 551:253 552:139 567:3 568:128 569:253 570:253 571:228 572:179 573:179 574:179 575:179 576:245 577:253 578:253 579:219 580:41 596:76 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:253 605:253 606:253 607:163 624:39 625:199 626:253 627:253 628:253 629:253 630:253 631:253 632:253 633:253 634:170 635:9 653:36 654:219 655:253 656:253 657:253 658:253 659:253 660:224 661:65 662:22 +1 156:202 157:253 158:69 184:253 185:252 186:121 212:253 213:252 214:69 240:253 241:252 242:69 267:106 268:253 269:231 270:37 295:179 296:255 297:196 322:17 323:234 324:253 325:92 350:93 351:252 352:253 353:92 378:93 379:252 380:253 381:92 406:93 407:252 408:232 409:8 434:208 435:253 436:116 462:207 463:252 464:116 490:207 491:252 492:32 517:57 518:244 519:252 545:122 546:252 547:252 573:185 574:253 575:253 601:184 602:252 603:252 629:101 630:252 631:252 657:13 658:173 659:252 660:43 686:9 687:232 688:116 +1 156:73 157:253 158:253 159:253 160:124 184:73 185:251 186:251 187:251 188:251 212:99 213:251 214:251 215:251 216:225 240:253 241:251 242:251 243:251 244:71 266:79 267:180 268:253 269:251 270:251 271:173 272:20 294:110 295:253 296:255 297:253 298:216 322:109 323:251 324:253 325:251 326:215 350:109 351:251 352:253 353:251 354:215 378:109 379:251 380:253 381:251 382:137 406:109 407:251 408:253 409:251 410:35 433:37 434:253 435:253 436:255 437:253 438:35 461:140 462:251 463:251 464:253 465:168 466:15 488:125 489:246 490:251 491:251 492:190 493:15 516:144 517:251 518:251 519:251 520:180 543:53 544:221 545:251 546:251 547:251 548:51 571:125 572:253 573:253 574:253 575:201 598:105 599:253 600:251 601:251 602:188 603:30 626:180 627:253 628:251 629:251 630:142 654:180 655:253 656:251 657:235 658:82 682:180 683:253 684:251 685:215 +1 124:111 125:255 126:48 152:162 153:253 154:237 155:63 180:206 181:253 182:253 183:183 208:87 209:217 210:253 211:205 237:90 238:253 239:238 240:60 265:37 266:225 267:253 268:89 294:206 295:253 296:159 322:206 323:253 324:226 350:206 351:253 352:226 378:206 379:253 380:226 406:206 407:253 408:226 434:206 435:253 436:226 462:206 463:253 464:226 490:206 491:253 492:226 518:206 519:253 520:237 521:45 546:206 547:253 548:253 549:109 574:173 575:253 576:253 577:109 602:69 603:253 604:253 605:109 630:64 631:248 632:253 633:109 659:112 660:253 661:109 +0 99:70 100:255 101:165 102:114 127:122 128:253 129:253 130:253 131:120 155:165 156:253 157:253 158:253 159:234 160:52 183:99 184:253 185:253 186:253 187:253 188:228 189:26 209:60 210:168 211:238 212:202 213:174 214:253 215:253 216:253 217:127 235:91 236:81 237:1 238:215 239:128 240:28 241:12 242:181 243:253 244:253 245:175 246:3 262:18 263:204 264:253 265:77 270:7 271:253 272:253 273:253 274:54 289:54 290:248 291:253 292:253 293:143 298:1 299:127 300:253 301:253 302:188 317:104 318:253 319:253 320:253 321:20 327:81 328:249 329:253 330:191 345:192 346:253 347:253 348:218 349:5 356:203 357:253 358:208 359:21 372:56 373:237 374:253 375:250 376:100 384:104 385:253 386:253 387:75 400:76 401:253 402:253 403:224 412:119 413:253 414:253 415:75 428:80 429:253 430:253 431:103 439:4 440:241 441:253 442:218 443:32 456:213 457:253 458:253 459:103 467:125 468:253 469:253 470:191 484:213 485:253 486:253 487:103 494:3 495:176 496:253 497:253 498:135 512:213 513:253 514:253 515:103 521:9 522:162 523:253 524:253 525:226 526:37 540:179 541:253 542:253 543:135 548:46 549:157 550:253 551:253 552:253 553:63 568:23 569:188 570:253 571:249 572:179 573:179 574:179 575:179 576:233 577:253 578:253 579:233 580:156 581:10 597:51 598:235 599:253 600:253 601:253 602:253 603:253 604:253 605:251 606:232 607:120 626:16 627:124 628:253 629:253 630:253 631:253 632:152 633:104 +1 124:29 125:197 126:255 127:84 152:85 153:251 154:253 155:83 180:86 181:253 182:254 183:253 208:85 209:251 210:253 211:251 236:86 237:253 238:254 239:253 240:169 264:85 265:251 266:253 267:251 268:168 292:86 293:253 294:254 295:253 296:169 320:28 321:196 322:253 323:251 324:168 349:169 350:254 351:253 352:169 377:168 378:253 379:251 380:168 405:169 406:254 407:253 408:169 433:168 434:253 435:251 436:168 462:254 463:253 464:254 465:139 490:253 491:251 492:253 493:251 518:254 519:253 520:254 521:253 522:57 546:253 547:251 548:253 549:251 550:168 574:198 575:253 576:254 577:253 578:114 602:85 603:251 604:253 605:251 630:85 631:253 632:254 633:253 658:28 659:83 660:196 661:83 +1 159:31 160:210 161:253 162:163 187:198 188:252 189:252 190:162 213:10 214:86 215:242 216:252 217:252 218:66 241:164 242:252 243:252 244:252 245:188 246:8 268:53 269:242 270:252 271:252 272:225 273:14 296:78 297:252 298:252 299:252 300:204 323:56 324:231 325:252 326:252 327:212 328:35 351:157 352:252 353:252 354:252 355:37 377:8 378:132 379:253 380:252 381:252 382:230 383:24 405:45 406:252 407:253 408:252 409:154 410:55 427:7 428:55 433:107 434:253 435:255 436:228 437:53 454:15 455:24 456:23 460:110 461:242 462:252 463:228 464:59 482:57 483:83 487:88 488:247 489:252 490:252 491:140 514:15 515:189 516:252 517:252 518:252 542:74 543:252 544:252 545:238 546:90 570:178 571:252 572:252 573:189 597:40 598:217 599:252 600:252 601:59 625:75 626:252 627:252 628:252 629:85 630:61 653:62 654:239 655:252 656:156 657:14 682:178 683:252 684:14 +1 131:159 132:255 133:122 158:167 159:228 160:253 161:121 185:64 186:236 187:251 188:205 189:110 212:48 213:158 214:251 215:251 216:178 217:39 240:190 241:251 242:251 243:251 267:96 268:253 269:253 270:253 271:153 295:194 296:251 297:251 298:211 299:74 322:80 323:174 324:251 325:251 326:140 327:47 349:16 350:181 351:253 352:251 353:219 354:23 377:64 378:251 379:253 380:251 381:204 382:19 405:223 406:253 407:255 408:233 409:48 431:20 432:174 433:244 434:251 435:253 436:109 437:31 459:96 460:189 461:251 462:251 463:126 464:31 486:24 487:106 488:251 489:235 490:188 491:100 514:96 515:251 516:251 517:228 518:59 542:255 543:253 544:253 545:213 546:36 569:100 570:253 571:251 572:251 573:85 574:23 596:32 597:127 598:253 599:235 600:126 601:15 624:104 625:251 626:253 627:240 628:79 652:83 653:193 654:253 655:220 +0 153:92 154:191 155:178 156:253 157:242 158:141 159:104 160:29 180:26 181:253 182:252 183:252 184:252 185:253 186:252 187:252 188:252 189:108 190:19 206:57 207:123 208:222 209:253 210:252 211:252 212:252 213:168 214:224 215:252 216:252 217:253 218:84 233:176 234:243 235:252 236:252 237:253 238:252 239:252 240:252 242:19 243:153 244:252 245:253 246:209 247:25 259:10 260:128 261:255 262:253 263:244 264:225 265:114 266:194 267:253 268:178 272:163 273:254 274:253 275:168 287:85 288:252 289:253 290:189 291:56 294:19 295:133 296:9 300:38 301:253 302:252 303:168 314:19 315:191 316:252 317:194 318:19 329:253 330:252 331:234 332:22 342:107 343:252 344:252 345:13 357:253 358:252 359:252 360:128 370:169 371:253 372:241 385:141 386:253 387:253 388:140 397:19 398:225 399:252 400:139 413:66 414:252 415:252 416:139 425:29 426:252 427:252 428:52 441:29 442:252 443:252 444:139 453:29 454:252 455:252 456:28 469:29 470:252 471:252 472:40 481:141 482:253 483:253 484:91 497:154 498:253 499:168 509:66 510:252 511:252 512:165 525:253 526:252 527:168 537:19 538:224 539:252 540:252 552:126 553:253 554:252 555:80 566:169 567:252 568:252 569:214 570:38 579:126 580:249 581:253 582:151 583:6 594:26 595:223 596:253 597:254 598:253 599:128 600:29 604:13 605:41 606:216 607:253 608:253 609:226 610:38 623:122 624:252 625:253 626:252 627:252 628:252 629:169 630:169 631:169 632:206 633:253 634:252 635:252 636:202 637:38 651:19 652:56 653:168 654:224 655:252 656:252 657:253 658:252 659:252 660:252 661:253 662:233 663:130 664:6 682:94 683:139 684:190 685:153 686:252 687:164 688:139 689:28 690:22 +1 128:53 129:250 130:255 131:25 156:167 157:253 158:253 159:25 182:3 183:123 184:247 185:253 186:253 187:25 210:9 211:253 212:253 213:253 214:253 215:25 238:9 239:253 240:253 241:253 242:253 243:25 266:9 267:253 268:253 269:253 270:180 271:13 294:9 295:253 296:253 297:253 298:104 322:9 323:253 324:253 325:253 326:104 350:15 351:253 352:253 353:253 354:104 378:184 379:253 380:253 381:228 382:68 406:184 407:253 408:253 409:182 433:103 434:251 435:253 436:253 437:12 461:106 462:253 463:253 464:253 465:8 488:24 489:238 490:253 491:253 492:253 493:8 516:27 517:253 518:253 519:253 520:253 521:8 544:27 545:253 546:253 547:253 548:253 549:8 572:27 573:253 574:253 575:253 576:177 577:4 600:160 601:253 602:253 603:253 604:87 628:202 629:253 630:253 631:219 632:54 656:81 657:253 658:247 659:51 +0 122:63 123:176 124:253 125:253 126:159 127:113 128:63 150:140 151:253 152:252 153:252 154:252 155:252 156:241 157:100 158:66 177:54 178:227 179:253 180:252 181:252 182:252 183:252 184:253 185:252 186:239 187:181 188:57 204:38 205:224 206:252 207:253 208:226 209:246 210:252 211:252 212:253 213:252 214:252 215:252 216:252 217:108 218:3 232:57 233:252 234:252 235:253 236:27 237:88 238:112 239:112 240:112 241:112 242:142 243:252 244:252 245:253 246:152 247:31 260:198 261:253 262:253 263:79 270:32 271:153 272:253 273:255 274:253 275:196 287:76 288:246 289:252 290:127 299:3 300:106 301:253 302:252 303:214 304:28 315:194 316:252 317:252 318:112 329:143 330:252 331:252 332:193 343:225 344:252 345:217 346:37 357:38 358:234 359:252 360:223 370:63 371:240 372:252 373:84 386:146 387:252 388:223 398:114 399:253 400:228 401:47 414:147 415:253 416:253 417:112 426:159 427:252 428:195 442:225 443:252 444:252 445:112 454:253 455:252 456:195 470:225 471:252 472:230 473:25 482:159 483:252 484:202 485:10 497:92 498:243 499:252 500:208 510:113 511:252 512:252 513:161 524:79 525:253 526:252 527:220 528:37 538:114 539:253 540:253 541:253 542:174 543:63 550:26 551:128 552:253 553:255 554:253 555:133 566:12 567:228 568:252 569:252 570:252 571:241 572:100 573:85 574:76 576:85 577:131 578:231 579:252 580:252 581:253 582:129 583:6 595:97 596:208 597:252 598:252 599:253 600:252 601:252 602:246 603:197 604:253 605:252 606:252 607:252 608:220 609:133 610:6 624:19 625:99 626:239 627:253 628:252 629:252 630:252 631:252 632:253 633:252 634:245 635:223 636:99 654:63 655:112 656:112 657:221 658:252 659:252 660:253 661:127 662:87 +0 153:12 154:136 155:254 156:255 157:195 158:115 159:3 180:6 181:175 182:253 183:196 184:160 185:252 186:253 187:15 208:130 209:253 210:234 211:4 213:27 214:205 215:232 216:40 235:54 236:246 237:253 238:68 242:24 243:243 244:106 262:3 263:134 264:235 265:99 266:4 271:132 272:247 273:77 290:56 291:253 292:62 299:23 300:233 301:129 318:179 319:183 320:4 328:182 329:220 345:21 346:232 347:59 356:95 357:232 358:21 373:128 374:183 385:228 386:85 401:187 402:124 413:228 414:186 429:187 430:124 441:228 442:104 457:187 458:124 469:169 470:184 485:187 486:124 497:203 498:150 513:187 514:124 524:10 525:220 526:39 541:187 542:155 552:111 553:201 569:129 570:228 571:7 579:12 580:181 581:76 598:234 599:166 600:9 606:24 607:209 608:106 626:139 627:250 628:167 629:11 630:2 631:11 632:11 633:129 634:227 635:90 636:11 655:95 656:247 657:253 658:178 659:253 660:253 661:244 662:86 684:47 685:175 686:253 687:232 688:149 689:40 +1 128:255 129:253 130:57 156:253 157:251 158:225 159:56 183:169 184:254 185:253 186:254 187:84 211:168 212:253 213:251 214:253 215:83 238:85 239:253 240:254 241:253 242:169 266:85 267:251 268:253 269:251 270:56 294:141 295:253 296:254 297:253 322:253 323:251 324:253 325:251 350:254 351:253 352:254 353:253 378:253 379:251 380:253 381:251 406:254 407:253 408:254 409:196 433:114 434:253 435:251 436:253 437:83 461:169 462:254 463:253 464:226 465:56 489:168 490:253 491:251 492:168 516:85 517:253 518:254 519:253 544:85 545:251 546:253 547:251 572:254 573:253 574:254 575:253 600:253 601:251 602:253 603:251 628:254 629:253 630:254 631:253 656:139 657:251 658:253 659:138 +0 151:23 152:167 153:208 154:254 155:255 156:129 157:19 179:151 180:253 181:253 182:253 183:253 184:253 185:209 186:26 207:181 208:253 209:253 210:253 211:227 212:181 213:253 214:207 215:22 235:227 236:253 237:253 238:253 239:92 240:38 241:226 242:253 243:129 244:2 263:193 264:253 265:253 266:248 267:62 269:50 270:253 271:253 272:45 291:170 292:253 293:253 294:135 297:12 298:208 299:253 300:119 318:16 319:232 320:253 321:253 322:21 326:60 327:253 328:185 346:164 347:253 348:253 349:224 350:14 354:14 355:217 356:247 357:62 373:3 374:193 375:253 376:250 377:64 383:199 384:253 385:179 401:67 402:253 403:253 404:205 411:98 412:253 413:188 429:151 430:253 431:245 432:43 439:63 440:250 441:188 457:151 458:253 459:243 468:244 469:222 470:22 485:151 486:253 487:217 496:244 497:253 498:115 512:3 513:195 514:253 515:134 524:156 525:253 526:150 541:140 542:253 543:134 552:239 553:253 554:139 569:44 570:253 571:134 579:53 580:246 581:237 582:32 597:8 598:200 599:229 600:40 606:25 607:225 608:253 609:188 626:120 627:250 628:230 629:58 630:17 632:12 633:42 634:213 635:253 636:238 637:84 655:151 656:253 657:253 658:217 659:179 660:206 661:253 662:253 663:196 664:118 683:18 684:58 685:145 686:152 687:253 688:214 689:145 690:74 691:7 +1 130:24 131:150 132:233 133:38 156:14 157:89 158:253 159:254 160:254 161:71 183:78 184:203 185:254 186:254 187:254 188:232 189:77 190:54 191:8 209:12 210:155 211:240 212:254 213:223 214:76 215:254 216:254 217:254 218:254 219:68 235:3 236:101 237:216 238:254 239:227 240:122 241:26 242:110 243:254 244:254 245:254 246:184 247:100 262:46 263:222 264:254 265:254 266:179 267:48 270:181 271:254 272:254 273:146 274:6 288:2 289:145 290:248 291:254 292:182 293:111 294:4 297:3 298:250 299:254 300:206 301:3 315:6 316:144 317:254 318:254 319:171 325:125 326:254 327:252 328:80 342:6 343:142 344:254 345:179 346:95 347:4 352:61 353:246 354:254 355:150 370:64 371:254 372:177 373:14 380:124 381:254 382:246 383:32 398:108 399:97 400:15 407:24 408:226 409:254 410:116 435:177 436:255 437:254 438:5 463:196 464:254 465:99 466:1 490:3 491:199 492:254 493:79 518:129 519:254 520:254 521:23 546:178 547:254 548:192 549:8 550:3 551:43 573:11 574:198 575:254 576:128 577:66 578:130 579:225 595:137 596:202 597:106 598:84 599:84 600:84 601:112 602:254 603:254 604:254 605:254 606:212 607:151 623:172 624:254 625:254 626:254 627:254 628:254 629:254 630:254 631:254 632:254 633:162 634:75 651:12 652:106 653:177 654:254 655:254 656:254 657:235 658:135 659:100 660:17 661:2 +0 125:120 126:253 127:253 128:63 151:38 152:131 153:246 154:252 155:252 156:203 157:15 179:222 180:252 181:252 182:252 183:252 184:166 185:38 205:4 206:107 207:253 208:252 209:252 210:252 211:252 212:253 213:224 214:137 215:26 233:107 234:252 235:253 236:252 237:220 238:128 239:252 240:253 241:252 242:252 243:239 244:140 261:170 262:253 263:255 264:168 267:79 268:192 269:253 270:253 271:253 272:253 273:255 274:90 288:51 289:243 290:252 291:215 292:33 296:12 297:74 298:233 299:252 300:252 301:253 302:195 303:19 316:166 317:252 318:252 319:31 326:43 327:149 328:195 329:253 330:252 331:177 332:19 343:57 344:234 345:252 346:252 357:237 358:252 359:252 360:180 361:13 371:85 372:252 373:252 374:173 385:50 386:237 387:252 388:252 389:112 399:226 400:253 401:240 402:63 414:163 415:253 416:253 417:112 426:38 427:234 428:252 429:176 442:85 443:252 444:252 445:158 454:113 455:252 456:252 457:84 470:19 471:209 472:252 473:252 482:207 483:252 484:252 485:84 498:10 499:203 500:252 501:236 510:253 511:252 512:252 513:84 526:85 527:252 528:252 529:112 538:114 539:253 540:253 541:146 553:51 554:159 555:253 556:240 557:63 566:75 567:243 568:252 569:249 570:146 579:57 580:85 581:238 582:252 583:252 584:99 595:116 596:252 597:252 598:252 599:198 600:197 601:165 602:57 603:57 604:57 605:182 606:197 607:234 608:252 609:253 610:233 611:164 612:19 623:28 624:84 625:180 626:252 627:253 628:252 629:252 630:252 631:252 632:253 633:252 634:252 635:252 636:252 637:225 638:71 653:13 654:112 655:253 656:252 657:252 658:252 659:252 660:253 661:252 662:252 663:157 664:112 +1 127:155 128:253 129:126 155:253 156:251 157:141 158:4 183:253 184:251 185:251 186:31 211:253 212:251 213:251 214:31 239:253 240:251 241:251 242:31 267:255 268:253 269:253 270:31 293:8 294:131 295:253 296:251 297:235 298:27 321:64 322:251 323:253 324:251 325:126 349:64 350:251 351:253 352:251 353:126 377:64 378:251 379:253 380:251 381:126 405:64 406:253 407:255 408:221 433:182 434:251 435:253 436:200 460:64 461:236 462:251 463:253 464:62 487:8 488:158 489:251 490:251 491:169 492:8 515:32 516:251 517:251 518:251 519:158 543:32 544:253 545:253 546:253 547:159 571:32 572:251 573:251 574:251 575:39 599:32 600:251 601:251 602:251 627:32 628:251 629:251 630:251 631:100 655:32 656:251 657:251 658:251 +0 101:88 102:127 103:5 126:19 127:58 128:20 129:14 130:217 131:19 152:7 153:146 154:247 155:253 156:235 157:27 158:84 159:81 180:126 181:253 182:164 183:19 184:15 187:156 188:9 208:214 209:222 210:34 215:234 216:58 235:59 236:254 237:116 243:235 244:58 263:141 264:251 265:72 271:151 272:140 291:224 292:233 299:136 300:223 319:254 320:218 327:136 328:253 347:254 348:135 355:136 356:253 374:23 375:255 376:114 383:137 384:231 402:98 403:254 404:122 411:136 412:155 430:98 431:254 432:106 439:166 440:155 458:98 459:254 460:128 467:234 468:193 486:98 487:254 488:135 494:61 495:248 496:118 515:255 516:238 517:18 521:13 522:224 523:254 524:58 543:201 544:253 545:128 546:2 548:5 549:150 550:253 551:167 552:9 571:18 572:226 573:253 574:49 575:31 576:156 577:253 578:228 579:13 600:147 601:253 602:243 603:241 604:254 605:227 606:43 628:5 629:126 630:245 631:253 632:231 633:46 +0 127:37 128:141 129:156 130:156 131:194 132:194 133:47 153:11 154:132 155:239 156:253 157:253 158:253 159:253 160:254 161:181 180:25 181:172 182:253 183:235 184:167 185:78 186:93 187:174 188:254 189:247 190:54 207:26 208:210 209:253 210:237 211:90 216:201 217:253 218:78 235:192 236:253 237:237 238:58 244:156 245:253 246:78 262:141 263:254 264:235 265:53 269:19 270:5 272:156 273:254 274:78 289:46 290:254 291:253 292:92 296:17 297:226 298:217 299:49 300:148 301:253 302:78 317:165 318:254 319:239 320:24 324:20 325:253 326:253 327:58 328:18 329:115 330:24 344:37 345:248 346:254 347:91 352:2 353:117 354:250 355:163 356:91 372:77 373:253 374:254 375:39 382:196 383:253 384:173 400:159 401:254 402:218 403:15 410:77 411:254 412:255 413:61 428:234 429:253 430:113 438:21 439:226 440:254 441:135 455:25 456:240 457:253 458:68 467:195 468:254 469:135 483:79 484:253 485:253 495:195 496:254 497:135 511:79 512:253 513:253 514:76 523:195 524:254 525:99 540:212 541:254 542:209 543:9 550:10 551:209 552:196 553:15 568:54 569:253 570:254 571:137 572:36 576:2 577:20 578:168 579:253 580:60 596:28 597:235 598:254 599:253 600:199 601:124 602:79 603:79 604:167 605:253 606:253 607:185 608:30 625:15 626:117 627:217 628:253 629:253 630:253 631:254 632:253 633:240 634:109 635:12 655:27 656:126 657:208 658:253 659:193 660:147 661:40 +0 154:32 155:134 156:218 157:254 158:254 159:254 160:217 161:84 176:44 177:208 178:215 179:156 180:35 181:119 182:236 183:246 184:136 185:91 186:69 187:151 188:249 189:246 190:78 203:44 204:230 205:254 206:254 207:254 208:254 209:254 210:196 211:48 216:60 217:224 218:210 219:24 231:118 232:254 233:202 234:19 235:201 236:254 237:181 238:9 245:35 246:233 247:168 259:193 260:223 261:34 263:59 264:163 265:236 266:15 274:140 275:205 276:8 286:60 287:254 288:176 293:38 302:54 303:237 304:80 314:59 315:254 316:93 331:131 332:200 342:59 343:240 344:24 359:79 360:214 370:59 371:234 387:67 388:248 389:54 398:59 399:234 416:235 417:58 426:60 427:235 443:79 444:255 445:59 454:59 455:251 456:66 471:79 472:250 473:54 482:59 483:254 484:108 499:146 500:214 510:5 511:203 512:187 513:3 526:4 527:188 528:199 539:118 540:254 541:57 554:96 555:254 556:117 567:16 568:237 569:224 570:14 581:14 582:187 583:206 584:8 596:88 597:252 598:186 599:16 608:16 609:187 610:252 611:125 625:100 626:254 627:237 628:94 629:24 635:13 636:214 637:254 638:166 653:3 654:57 655:215 656:248 657:241 658:235 659:197 660:137 661:137 662:137 663:231 664:238 665:155 666:25 684:57 685:155 686:246 687:254 688:254 689:254 690:254 691:147 692:36 +1 124:102 125:252 126:252 127:41 152:102 153:250 154:250 155:202 180:102 181:250 182:250 183:232 184:91 208:102 209:250 210:250 211:212 212:29 236:102 237:252 238:252 239:254 240:150 264:102 265:250 266:250 267:252 268:149 292:102 293:250 294:250 295:252 296:149 320:102 321:250 322:250 323:252 324:231 325:80 349:152 350:252 351:254 352:252 353:100 377:151 378:250 379:252 380:250 381:100 405:151 406:250 407:252 408:250 409:100 433:151 434:250 435:252 436:250 437:100 461:123 462:243 463:254 464:252 465:100 490:202 491:252 492:250 493:100 518:80 519:252 520:250 521:190 522:30 547:252 548:250 549:250 550:49 575:255 576:252 577:252 578:252 579:214 580:31 603:171 604:250 605:250 606:250 607:252 608:190 609:40 631:20 632:160 633:250 634:250 635:252 636:250 637:100 660:20 661:170 662:250 663:212 664:49 665:20 +0 124:20 125:121 126:197 127:253 128:64 151:23 152:200 153:252 154:252 155:252 156:184 157:6 178:25 179:197 180:252 181:252 182:252 183:252 184:253 185:228 186:107 187:15 205:26 206:196 207:252 208:252 209:252 210:252 211:252 212:253 213:252 214:252 215:219 216:178 217:21 233:186 234:252 235:238 236:94 237:67 238:224 239:217 240:53 241:109 242:245 243:252 244:252 245:213 246:63 260:98 261:242 262:252 263:101 266:39 267:31 270:109 271:128 272:241 273:252 274:207 275:97 287:17 288:230 289:252 290:241 291:56 300:109 301:252 302:252 303:229 304:17 314:13 315:192 316:252 317:243 318:96 328:25 329:127 330:252 331:252 332:120 342:121 343:252 344:252 345:165 357:125 358:252 359:252 360:246 361:70 370:190 371:252 372:252 373:39 385:26 386:210 387:252 388:252 389:119 398:255 399:253 400:159 414:22 415:209 416:253 417:183 426:253 427:252 428:103 443:34 444:252 445:252 454:253 455:252 456:26 471:27 472:252 473:252 482:253 483:252 484:168 485:13 499:70 500:252 501:209 510:147 511:252 512:252 513:75 526:68 527:233 528:252 529:119 538:121 539:252 540:252 541:189 542:40 552:15 553:82 554:231 555:252 556:214 557:31 566:38 567:135 568:248 569:252 570:231 571:145 572:41 573:41 574:41 575:41 576:20 577:24 578:37 579:83 580:194 581:252 582:252 583:212 584:33 596:83 597:213 598:252 599:252 600:252 601:252 602:252 603:252 604:204 605:213 606:243 607:252 608:252 609:252 610:212 611:34 625:34 626:140 627:238 628:248 629:252 630:252 631:252 632:253 633:252 634:252 635:241 636:238 637:238 638:75 656:82 657:119 658:119 659:119 660:120 661:119 662:119 663:19 +1 127:20 128:254 129:255 130:37 155:19 156:253 157:253 158:134 183:19 184:253 185:253 186:246 187:125 211:76 212:253 213:253 214:253 215:158 239:207 240:253 241:253 242:253 243:158 267:207 268:253 269:253 270:253 271:158 294:48 295:223 296:253 297:253 298:243 299:106 322:141 323:253 324:253 325:253 326:113 349:65 350:237 351:253 352:253 353:253 354:36 377:76 378:253 379:253 380:253 381:253 382:36 405:76 406:253 407:253 408:253 409:253 410:36 433:76 434:253 435:253 436:253 437:118 438:4 460:4 461:148 462:253 463:253 464:253 465:103 488:10 489:253 490:253 491:253 492:253 493:103 516:10 517:253 518:253 519:253 520:173 521:7 544:10 545:253 546:253 547:253 548:168 572:143 573:253 574:253 575:239 576:49 600:198 601:253 602:253 603:234 615:140 628:198 629:253 630:253 631:234 656:198 657:253 658:253 659:234 +0 235:40 236:37 238:7 239:77 240:137 241:136 242:136 243:136 244:136 245:40 246:6 261:16 262:135 263:254 264:233 266:152 267:215 268:96 269:140 270:155 271:118 272:230 273:254 274:158 275:68 288:19 289:164 290:254 291:114 294:235 295:140 301:99 302:230 303:254 304:186 305:14 315:70 316:226 317:242 318:121 322:104 323:195 324:38 330:33 331:179 332:253 333:140 342:41 343:241 344:198 345:43 359:24 360:209 361:223 370:164 371:250 372:66 388:136 389:253 398:254 399:158 416:136 417:215 426:255 427:76 442:5 443:127 444:246 445:133 454:254 455:122 469:5 470:150 471:247 472:91 473:9 482:254 483:165 495:13 496:79 497:194 498:216 499:84 510:111 511:251 512:87 519:16 520:25 521:40 522:107 523:186 524:213 525:117 526:25 538:14 539:185 540:235 541:142 542:23 546:91 547:157 548:231 549:207 550:126 551:49 569:143 570:195 571:255 572:254 573:254 574:244 575:157 576:76 599:39 600:39 601:39 602:33 +1 128:166 129:255 130:187 131:6 156:165 157:253 158:253 159:13 183:15 184:191 185:253 186:253 187:13 211:49 212:253 213:253 214:253 215:13 239:141 240:253 241:253 242:169 243:4 266:4 267:189 268:253 269:249 270:53 294:69 295:253 296:253 297:246 322:69 323:253 324:253 325:246 350:118 351:253 352:253 353:124 378:206 379:253 380:231 381:21 405:66 406:241 407:253 408:199 433:105 434:253 435:253 436:89 460:3 461:228 462:253 463:252 464:86 488:111 489:253 490:253 491:205 516:166 517:253 518:253 519:75 543:43 544:249 545:253 546:193 547:9 570:4 571:160 572:253 573:253 574:184 598:37 599:253 600:253 601:253 602:88 626:140 627:253 628:253 629:186 630:18 654:14 655:253 656:253 657:27 +1 128:117 129:128 155:2 156:199 157:127 183:81 184:254 185:87 211:116 212:254 213:48 239:175 240:234 241:18 266:5 267:230 268:168 294:80 295:255 296:142 322:80 323:255 324:142 350:80 351:251 352:57 378:129 379:239 406:164 407:209 433:28 434:245 435:159 461:64 462:254 463:144 489:84 490:254 491:80 517:143 518:254 519:30 544:3 545:225 546:200 572:48 573:254 574:174 600:48 601:254 602:174 628:93 629:254 630:129 656:53 657:234 658:41 +1 129:159 130:142 156:11 157:220 158:141 184:78 185:254 186:141 212:111 213:254 214:109 240:196 241:221 242:15 267:26 268:221 269:159 295:63 296:254 297:159 323:178 324:254 325:93 350:7 351:191 352:254 353:97 378:42 379:255 380:254 381:41 406:42 407:254 408:195 409:10 434:141 435:255 436:78 461:11 462:202 463:254 464:59 489:86 490:254 491:254 492:59 517:142 518:254 519:248 520:52 545:142 546:254 547:195 573:142 574:254 575:164 601:142 602:254 603:77 629:142 630:254 631:131 657:77 658:172 659:5 +0 124:66 125:254 126:254 127:58 128:60 129:59 130:59 131:50 151:73 152:233 153:253 154:253 155:148 156:254 157:253 158:253 159:232 160:73 179:156 180:253 181:253 182:253 183:117 184:255 185:253 186:253 187:253 188:223 189:176 190:162 205:37 206:116 207:246 208:253 209:180 210:18 211:4 212:18 213:109 214:241 215:253 216:253 217:253 218:236 219:28 233:235 234:253 235:253 236:245 237:107 242:109 243:170 244:253 245:253 246:253 247:174 261:235 262:253 263:253 264:233 271:15 272:156 273:253 274:253 275:223 276:72 287:10 288:156 289:250 290:253 291:253 292:67 300:99 301:253 302:253 303:253 304:127 305:5 315:118 316:253 317:253 318:253 319:204 320:26 328:68 329:223 330:253 331:253 332:253 333:57 342:32 343:191 344:253 345:253 346:253 347:97 357:156 358:253 359:253 360:253 361:57 370:59 371:253 372:253 373:253 374:253 375:97 385:36 386:224 387:253 388:253 389:57 398:60 399:254 400:255 401:254 402:156 413:37 414:226 415:254 416:254 417:58 426:59 427:253 428:253 429:253 430:154 441:156 442:253 443:253 444:253 445:57 454:59 455:253 456:253 457:253 458:154 469:156 470:253 471:253 472:253 473:57 482:59 483:253 484:253 485:253 486:246 487:90 496:16 497:171 498:253 499:253 500:231 501:49 510:59 511:253 512:253 513:253 514:253 515:156 516:91 524:99 525:253 526:253 527:222 528:71 538:59 539:253 540:253 541:253 542:253 543:253 544:245 545:109 551:145 552:194 553:253 554:253 555:174 566:9 567:38 568:174 569:251 570:253 571:253 572:253 573:241 574:215 575:215 576:217 577:215 578:215 579:250 580:253 581:253 582:221 583:26 597:235 598:253 599:253 600:253 601:253 602:253 603:253 604:254 605:253 606:253 607:253 608:253 609:204 610:26 625:108 626:116 627:200 628:253 629:253 630:253 631:253 632:254 633:253 634:253 635:253 636:199 637:44 655:36 656:57 657:118 658:253 659:253 660:58 661:57 662:57 663:57 664:35 +1 129:101 130:222 131:84 157:225 158:252 159:84 184:89 185:246 186:208 187:19 212:128 213:252 214:195 239:79 240:253 241:252 242:195 267:141 268:255 269:253 270:133 294:26 295:240 296:253 297:252 298:55 322:60 323:252 324:253 325:154 326:12 349:7 350:178 351:252 352:253 353:27 377:57 378:252 379:252 380:253 381:27 405:57 406:253 407:253 408:204 409:15 433:104 434:252 435:252 436:94 460:19 461:209 462:252 463:252 488:101 489:252 490:252 491:157 516:225 517:252 518:252 519:112 544:226 545:253 546:240 547:63 572:225 573:252 574:223 600:225 601:252 602:223 628:225 629:252 630:242 631:75 656:146 657:252 658:236 659:50 +0 124:41 125:254 126:254 127:157 128:34 129:34 130:218 131:255 132:206 133:34 134:18 151:53 152:238 153:252 154:252 155:252 156:252 157:252 158:252 159:252 160:252 161:252 162:162 163:26 178:66 179:220 180:252 181:252 182:252 183:209 184:153 185:223 186:252 187:252 188:252 189:252 190:252 191:98 206:166 207:252 208:252 209:252 210:252 211:141 213:85 214:230 215:252 216:252 217:252 218:252 219:98 234:166 235:252 236:252 237:252 238:252 239:141 242:73 243:102 244:252 245:252 246:252 247:98 262:166 263:252 264:252 265:252 266:191 267:30 271:5 272:97 273:252 274:252 275:220 276:51 289:123 290:245 291:252 292:252 293:202 294:14 300:56 301:252 302:252 303:252 304:65 316:18 317:154 318:252 319:252 320:241 328:56 329:252 330:252 331:252 332:65 343:21 344:146 345:252 346:252 347:252 348:241 356:56 357:252 358:252 359:252 360:65 371:67 372:252 373:252 374:252 375:252 376:241 384:56 385:252 386:252 387:252 388:65 399:67 400:252 401:252 402:252 403:252 404:116 412:56 413:252 414:252 415:252 416:65 427:67 428:252 429:252 430:252 431:252 432:20 440:56 441:252 442:252 443:252 444:65 455:67 456:252 457:252 458:252 459:87 460:4 468:56 469:252 470:252 471:124 472:11 483:67 484:252 485:252 486:252 487:54 494:19 495:236 496:245 497:252 498:252 499:98 511:67 512:252 513:252 514:252 515:97 516:5 521:39 522:219 523:252 524:252 525:252 526:252 527:98 539:67 540:252 541:252 542:252 543:252 544:102 545:89 546:89 547:89 548:89 549:203 550:252 551:252 552:252 553:252 554:209 555:64 567:67 568:252 569:252 570:252 571:252 572:252 573:252 574:252 575:252 576:252 577:252 578:252 579:252 580:226 581:130 582:68 595:67 596:252 597:252 598:252 599:252 600:252 601:252 602:252 603:252 604:252 605:252 606:252 607:239 608:77 623:17 624:65 625:163 626:252 627:252 628:252 629:252 630:252 631:252 632:252 633:252 634:96 635:59 653:17 654:176 655:252 656:252 657:252 658:252 659:155 660:32 661:32 662:6 +0 96:56 97:247 98:121 124:24 125:242 126:245 127:122 153:231 154:253 155:253 156:104 157:12 181:90 182:253 183:253 184:254 185:221 186:120 187:120 188:85 206:67 207:75 208:36 209:11 210:56 211:222 212:254 213:253 214:253 215:253 216:245 217:207 218:36 233:86 234:245 235:249 236:105 239:44 240:224 241:230 242:253 243:253 244:253 245:253 246:214 247:10 260:8 261:191 262:253 263:143 269:29 270:119 271:119 272:158 273:253 274:253 275:94 288:15 289:253 290:226 291:48 300:4 301:183 302:253 303:248 304:56 316:42 317:253 318:178 329:179 330:253 331:184 332:14 344:164 345:253 346:178 357:179 358:253 359:163 371:61 372:254 373:254 374:179 384:76 385:254 386:254 387:164 399:60 400:253 401:253 402:178 411:29 412:206 413:253 414:253 415:40 427:60 428:253 429:253 430:178 439:120 440:253 441:253 442:245 443:13 455:60 456:253 457:253 458:178 467:120 468:253 469:239 470:63 483:60 484:253 485:253 486:178 494:14 495:238 496:253 497:179 511:18 512:190 513:253 514:231 515:70 521:43 522:184 523:253 524:253 525:74 540:86 541:253 542:253 543:239 544:134 545:8 548:56 549:163 550:253 551:253 552:213 553:35 568:16 569:253 570:253 571:253 572:253 573:240 574:239 575:239 576:247 577:253 578:253 579:210 580:27 596:4 597:59 598:204 599:253 600:253 601:253 602:253 603:253 604:254 605:253 606:250 607:110 626:31 627:122 628:253 629:253 630:253 631:253 632:255 633:217 634:98 +0 125:19 126:164 127:253 128:255 129:253 130:118 131:59 132:36 153:78 154:251 155:251 156:253 157:251 158:251 159:251 160:199 161:45 180:14 181:198 182:251 183:251 184:253 185:251 186:251 187:251 188:251 189:204 190:26 208:5 209:117 210:251 211:251 212:243 213:212 214:239 215:251 216:251 217:251 218:218 236:95 237:251 238:251 239:251 240:120 242:175 243:251 244:251 245:251 246:231 263:97 264:237 265:251 266:251 267:251 270:67 271:240 272:251 273:251 274:243 275:108 290:8 291:163 292:251 293:251 294:240 295:81 299:68 300:251 301:251 302:251 303:179 304:9 317:13 318:145 319:251 320:251 321:226 322:80 327:39 328:251 329:251 330:251 331:251 332:115 345:144 346:251 347:251 348:251 349:173 355:18 356:167 357:251 358:251 359:251 360:115 373:233 374:251 375:251 376:251 377:173 384:98 385:251 386:251 387:251 388:115 400:176 401:253 402:253 403:216 404:179 412:99 413:253 414:253 415:253 416:116 427:55 428:210 429:251 430:251 431:96 440:98 441:251 442:251 443:214 444:62 455:117 456:251 457:251 458:251 459:96 467:28 468:204 469:251 470:237 471:53 482:55 483:241 484:251 485:251 486:160 487:7 494:28 495:222 496:251 497:251 498:231 510:59 511:251 512:251 513:251 514:153 520:23 521:98 522:204 523:251 524:251 525:251 526:156 538:59 539:251 540:251 541:251 542:153 546:85 547:155 548:179 549:251 550:251 551:251 552:251 553:154 554:15 566:59 567:251 568:251 569:251 570:236 571:214 572:214 573:214 574:234 575:251 576:253 577:251 578:251 579:248 580:156 581:15 594:41 595:209 596:251 597:251 598:251 599:251 600:251 601:251 602:251 603:251 604:253 605:251 606:196 607:146 623:54 624:115 625:241 626:251 627:251 628:251 629:251 630:251 631:251 632:253 633:187 634:35 653:83 654:251 655:251 656:251 657:251 658:251 659:101 660:57 661:31 +1 129:232 130:255 131:107 156:58 157:244 158:253 159:106 184:95 185:253 186:253 187:106 212:95 213:253 214:253 215:106 240:95 241:253 242:249 243:69 268:144 269:253 270:192 295:97 296:233 297:253 298:66 323:195 324:253 325:253 326:5 350:38 351:232 352:253 353:182 354:2 377:10 378:160 379:253 380:231 381:53 405:42 406:253 407:253 408:158 433:141 434:253 435:253 436:115 460:75 461:245 462:253 463:183 464:4 487:1 488:147 489:253 490:251 491:58 515:20 516:253 517:253 518:180 543:202 544:253 545:226 546:27 571:243 572:253 573:212 598:85 599:251 600:253 601:173 626:209 627:253 628:244 629:57 654:169 655:253 656:174 +1 127:63 128:128 129:2 155:63 156:254 157:123 183:63 184:254 185:179 211:63 212:254 213:179 239:63 240:254 241:179 267:142 268:254 269:179 295:187 296:254 297:158 323:187 324:254 325:55 350:68 351:235 352:254 353:55 378:181 379:254 380:254 381:55 406:181 407:254 408:202 409:14 434:181 435:254 436:186 462:181 463:254 464:146 490:181 491:254 492:62 518:181 519:254 520:62 546:181 547:254 548:62 574:181 575:255 576:62 602:181 603:254 604:241 605:52 630:181 631:254 632:222 633:30 658:181 659:224 660:34 +1 130:131 131:255 132:184 133:15 157:99 158:247 159:253 160:182 161:15 185:124 186:253 187:253 188:253 189:38 212:9 213:171 214:253 215:253 216:140 217:1 240:47 241:253 242:253 243:251 244:117 267:43 268:219 269:253 270:253 271:153 295:78 296:253 297:253 298:253 299:84 323:97 324:253 325:253 326:244 327:74 350:69 351:243 352:253 353:253 354:183 377:10 378:168 379:253 380:253 381:215 382:34 405:31 406:253 407:253 408:253 409:129 433:107 434:253 435:253 436:242 437:67 460:24 461:204 462:253 463:253 464:187 488:95 489:253 490:253 491:201 492:25 516:239 517:253 518:253 519:176 543:119 544:251 545:253 546:253 547:138 570:30 571:212 572:253 573:252 574:165 575:8 598:193 599:253 600:253 601:222 626:193 627:253 628:253 629:189 654:193 655:253 656:201 657:27 +0 125:57 126:255 127:253 128:198 129:85 153:168 154:253 155:251 156:253 157:251 158:169 159:56 180:86 181:253 182:254 183:253 184:254 185:253 186:254 187:253 188:57 208:197 209:251 210:253 211:251 212:253 213:251 214:253 215:251 216:225 217:56 235:169 236:255 237:253 238:226 239:56 241:114 242:254 243:253 244:254 245:84 262:57 263:224 264:253 265:251 266:56 270:139 271:251 272:253 273:83 290:141 291:253 292:255 293:84 298:57 299:225 300:254 301:196 318:253 319:251 320:253 321:83 327:168 328:253 329:83 345:169 346:254 347:253 348:169 355:169 356:254 357:253 358:169 373:168 374:253 375:251 376:56 383:168 384:253 385:251 386:56 401:169 402:254 403:84 412:254 413:253 429:168 430:253 431:83 440:253 441:251 456:29 457:197 458:254 459:84 467:169 468:254 469:196 484:85 485:251 486:253 487:83 494:57 495:224 496:253 497:83 512:57 513:225 514:254 515:139 521:57 522:141 523:253 524:254 525:84 541:168 542:253 543:251 544:169 545:56 547:114 548:169 549:224 550:253 551:251 552:253 553:83 569:169 570:254 571:253 572:254 573:253 574:254 575:253 576:254 577:253 578:254 579:253 580:226 581:56 597:56 598:253 599:251 600:253 601:251 602:253 603:251 604:253 605:251 606:253 607:251 608:56 626:169 627:225 628:254 629:253 630:254 631:253 632:254 633:253 634:226 635:56 655:56 656:253 657:251 658:253 659:251 660:84 661:83 662:56 +0 127:12 128:105 129:224 130:255 131:247 132:22 155:131 156:254 157:254 158:243 159:252 160:76 182:131 183:225 184:254 185:224 186:48 187:136 208:13 209:109 210:252 211:254 212:254 213:254 214:197 215:76 235:9 236:181 237:254 238:254 239:240 240:229 241:237 242:254 243:252 244:152 245:21 262:9 263:143 264:254 265:254 266:226 267:36 269:22 270:138 271:254 272:254 273:188 289:13 290:181 291:254 292:254 293:250 294:64 298:2 299:53 300:236 301:252 302:131 317:102 318:254 319:254 320:254 321:111 328:56 329:243 330:251 331:42 344:30 345:186 346:254 347:254 348:206 349:29 357:199 358:254 359:91 372:92 373:254 374:254 375:237 376:13 385:134 386:254 387:91 400:133 401:254 402:254 403:126 413:134 414:250 415:17 428:187 429:254 430:237 431:23 441:200 442:183 456:187 457:254 458:213 467:2 468:134 469:252 470:101 484:183 485:254 486:133 495:14 496:254 497:234 498:34 512:92 513:254 514:161 522:84 523:204 524:254 525:56 540:92 541:254 542:229 549:85 550:252 551:252 552:188 553:11 568:56 569:252 570:229 575:3 576:53 577:235 578:253 579:166 597:224 598:245 599:130 600:68 601:68 602:134 603:214 604:254 605:254 606:159 625:141 626:254 627:254 628:254 629:254 630:254 631:254 632:233 633:95 634:3 653:14 654:152 655:254 656:254 657:254 658:186 659:157 660:53 +1 130:226 131:247 132:55 157:99 158:248 159:254 160:230 161:30 185:125 186:254 187:254 188:254 189:38 213:125 214:254 215:254 216:212 217:24 240:18 241:223 242:254 243:252 244:118 268:24 269:254 270:254 271:239 295:27 296:195 297:254 298:254 299:93 323:78 324:254 325:254 326:246 327:74 351:158 352:254 353:254 354:185 378:41 379:239 380:254 381:254 382:43 405:22 406:218 407:254 408:254 409:167 410:9 433:32 434:254 435:254 436:254 437:130 460:24 461:187 462:254 463:254 464:234 465:16 488:189 489:254 490:254 491:254 492:128 515:64 516:247 517:254 518:255 519:219 520:42 543:139 544:254 545:254 546:222 547:40 570:30 571:213 572:254 573:235 574:45 598:194 599:254 600:254 601:223 626:194 627:254 628:254 629:190 654:194 655:254 656:202 657:27 +1 130:166 131:253 132:124 133:53 158:140 159:251 160:251 161:180 185:125 186:246 187:251 188:251 189:51 212:32 213:190 214:251 215:251 216:251 217:103 240:21 241:174 242:251 243:251 244:251 268:73 269:176 270:253 271:253 272:201 296:149 297:251 298:251 299:251 300:71 323:27 324:228 325:251 326:251 327:157 328:10 351:180 352:253 353:251 354:251 355:142 377:27 378:180 379:231 380:253 381:251 382:96 383:41 405:89 406:253 407:253 408:255 409:211 410:25 433:217 434:251 435:251 436:253 437:107 460:21 461:221 462:251 463:251 464:242 465:92 487:32 488:190 489:251 490:251 491:251 492:103 515:202 516:251 517:251 518:251 519:122 542:53 543:255 544:253 545:253 546:221 547:51 570:180 571:253 572:251 573:251 574:142 598:180 599:253 600:251 601:251 602:142 626:180 627:253 628:251 629:157 630:82 654:180 655:253 656:147 657:10 +1 129:17 130:206 131:229 132:44 157:2 158:125 159:254 160:123 185:95 186:254 187:254 188:123 212:78 213:240 214:254 215:254 216:123 240:100 241:254 242:254 243:254 244:123 267:2 268:129 269:254 270:254 271:220 272:20 295:9 296:254 297:254 298:254 299:123 322:22 323:179 324:254 325:254 326:254 327:49 350:83 351:254 352:254 353:254 354:183 355:19 378:136 379:254 380:254 381:254 382:139 404:3 405:111 406:252 407:254 408:254 409:232 410:45 432:67 433:254 434:254 435:254 436:216 437:40 459:14 460:192 461:254 462:254 463:254 464:140 486:23 487:192 488:254 489:254 490:254 491:246 514:77 515:254 516:254 517:255 518:241 519:100 541:65 542:235 543:254 544:254 545:254 546:172 568:30 569:238 570:254 571:254 572:254 573:219 574:26 596:34 597:254 598:254 599:254 600:216 601:41 624:34 625:254 626:254 627:254 628:188 652:12 653:170 654:254 655:254 656:82 +1 130:218 131:253 132:124 157:84 158:236 159:251 160:251 184:63 185:236 186:251 187:251 188:122 212:73 213:251 214:251 215:251 216:173 240:202 241:251 242:251 243:251 244:71 267:53 268:255 269:253 270:253 271:253 272:72 295:180 296:253 297:251 298:251 299:188 300:30 323:180 324:253 325:251 326:251 327:142 350:47 351:211 352:253 353:251 354:235 355:82 377:27 378:211 379:251 380:253 381:251 382:215 405:89 406:253 407:253 408:255 409:253 410:164 433:217 434:251 435:251 436:253 437:168 438:15 460:21 461:221 462:251 463:251 464:253 465:107 487:32 488:190 489:251 490:251 491:251 492:221 493:61 515:73 516:251 517:251 518:251 519:251 520:180 543:255 544:253 545:253 546:253 547:201 570:105 571:253 572:251 573:251 574:251 575:71 598:180 599:253 600:251 601:246 602:137 603:10 626:180 627:253 628:251 629:215 654:180 655:253 656:251 657:86 +1 124:102 125:180 126:1 152:140 153:254 154:130 180:140 181:254 182:204 208:140 209:254 210:204 236:72 237:254 238:204 264:25 265:231 266:250 267:135 292:11 293:211 294:254 295:222 321:101 322:254 323:250 324:15 349:96 350:254 351:254 352:95 377:2 378:251 379:254 380:95 405:2 406:251 407:254 408:95 433:96 434:254 435:254 436:95 461:53 462:253 463:254 464:139 490:250 491:254 492:235 493:27 518:201 519:254 520:254 521:128 546:80 547:254 548:254 549:139 574:65 575:254 576:254 577:139 602:150 603:254 604:254 605:139 630:229 631:254 632:254 633:43 658:52 659:196 660:168 661:9 +0 128:87 129:208 130:249 155:27 156:212 157:254 158:195 182:118 183:225 184:254 185:254 186:232 187:147 188:46 209:115 210:248 211:254 212:254 213:254 214:254 215:254 216:230 217:148 218:12 236:18 237:250 238:254 239:245 240:226 241:254 242:254 243:254 244:254 245:254 246:148 263:92 264:205 265:254 266:250 267:101 268:20 269:194 270:254 271:254 272:254 273:254 274:229 275:53 291:152 292:254 293:254 294:94 297:14 298:124 299:187 300:254 301:254 302:254 303:213 318:95 319:252 320:254 321:206 322:15 327:3 328:6 329:51 330:231 331:254 332:94 345:50 346:246 347:254 348:254 349:20 358:200 359:254 360:96 372:21 373:184 374:254 375:254 376:147 377:2 386:200 387:254 388:96 400:177 401:254 402:254 403:218 404:33 413:16 414:211 415:254 416:96 427:11 428:219 429:254 430:251 431:92 441:84 442:254 443:232 444:44 455:101 456:254 457:254 458:141 469:162 470:254 471:231 472:42 483:235 484:254 485:227 486:42 496:51 497:238 498:254 499:213 511:235 512:254 513:199 524:160 525:254 526:229 527:52 539:235 540:254 541:199 549:10 550:84 551:150 552:253 553:254 554:147 567:235 568:254 569:213 570:20 575:17 576:63 577:158 578:254 579:254 580:254 581:155 582:12 595:122 596:248 597:254 598:204 599:98 600:42 601:177 602:180 603:200 604:254 605:254 606:253 607:213 608:82 609:10 624:203 625:254 626:254 627:254 628:254 629:254 630:254 631:254 632:251 633:219 634:94 652:35 653:221 654:254 655:254 656:254 657:254 658:254 659:217 660:95 +1 126:134 127:230 154:133 155:231 156:10 182:133 183:253 184:96 210:133 211:253 212:96 238:133 239:253 240:183 266:133 267:253 268:217 294:133 295:253 296:217 322:133 323:253 324:217 350:133 351:253 352:217 378:133 379:253 380:217 406:134 407:254 408:218 434:133 435:253 436:159 462:133 463:253 464:199 490:156 491:253 492:96 518:254 519:247 520:73 546:254 547:248 548:74 573:99 574:254 575:245 576:64 600:89 601:230 602:254 603:125 627:140 628:251 629:253 630:243 631:10 655:114 656:242 657:195 658:69 +1 125:29 126:85 127:255 128:139 153:197 154:251 155:253 156:251 181:254 182:253 183:254 184:253 209:253 210:251 211:253 212:251 237:254 238:253 239:254 240:253 265:253 266:251 267:253 268:138 293:254 294:253 295:254 296:196 321:253 322:251 323:253 324:196 349:254 350:253 351:254 352:84 377:253 378:251 379:253 380:196 405:254 406:253 407:254 408:253 433:253 434:251 435:253 436:251 461:254 462:253 463:254 464:253 489:253 490:251 491:253 492:251 517:254 518:253 519:254 520:253 545:253 546:251 547:253 548:251 573:254 574:253 575:254 576:253 601:253 602:251 603:253 604:251 629:57 630:225 631:254 632:253 658:56 659:253 660:251 +1 125:149 126:255 127:254 128:58 153:215 154:253 155:183 156:2 180:41 181:232 182:253 183:181 208:92 209:253 210:253 211:181 236:92 237:253 238:253 239:181 264:92 265:253 266:253 267:181 292:92 293:253 294:253 295:181 320:92 321:253 322:253 323:181 348:92 349:253 350:253 351:181 376:92 377:253 378:253 379:181 404:92 405:253 406:253 407:181 432:92 433:253 434:253 435:181 460:92 461:253 462:253 463:181 488:31 489:228 490:253 491:181 517:198 518:253 519:228 520:54 545:33 546:226 547:253 548:195 549:7 574:199 575:253 576:253 577:75 602:34 603:218 604:253 605:228 606:117 607:14 608:12 631:33 632:219 633:253 634:253 635:253 636:211 660:32 661:123 662:149 663:230 664:41 +1 130:79 131:203 132:141 157:51 158:240 159:240 160:140 185:88 186:252 187:252 188:140 213:197 214:252 215:252 216:140 241:197 242:252 243:252 244:140 268:147 269:253 270:253 271:253 295:38 296:234 297:252 298:242 299:89 323:113 324:252 325:252 326:223 350:16 351:207 352:252 353:252 354:129 377:16 378:203 379:253 380:252 381:220 382:37 405:29 406:253 407:255 408:253 409:56 432:19 433:181 434:252 435:253 436:176 437:6 460:166 461:252 462:252 463:228 464:52 487:10 488:203 489:252 490:252 491:126 514:63 515:178 516:252 517:252 518:173 542:114 543:253 544:253 545:225 570:238 571:252 572:252 573:99 596:7 597:135 598:253 599:252 600:176 601:19 624:29 625:252 626:253 627:252 628:55 652:13 653:189 654:253 655:204 656:25 +1 126:94 127:254 128:75 154:166 155:253 156:231 182:208 183:253 184:147 210:208 211:253 212:116 238:208 239:253 240:168 266:146 267:254 268:222 294:166 295:253 296:116 322:208 323:253 324:116 350:166 351:253 352:158 378:145 379:253 380:231 406:209 407:254 408:169 434:187 435:253 436:168 462:93 463:253 464:116 490:93 491:253 492:116 518:93 519:253 520:116 546:94 547:254 548:179 549:11 574:93 575:253 576:246 577:101 602:145 603:253 604:255 605:92 630:93 631:253 632:246 633:59 658:93 659:253 660:74 +0 127:46 128:105 129:254 130:254 131:224 132:59 133:59 134:9 155:196 156:254 157:253 158:253 159:253 160:253 161:253 162:128 182:96 183:235 184:254 185:253 186:253 187:253 188:253 189:253 190:247 191:122 208:4 209:101 210:244 211:253 212:254 213:234 214:241 215:253 216:253 217:253 218:253 219:186 220:18 236:96 237:253 238:253 239:253 240:232 241:83 242:109 243:170 244:253 245:253 246:253 247:253 248:116 264:215 265:253 266:253 267:253 268:196 271:40 272:253 273:253 274:253 275:253 276:116 290:8 291:141 292:247 293:253 294:253 295:237 296:29 299:6 300:38 301:171 302:253 303:253 304:116 317:13 318:146 319:253 320:253 321:253 322:253 323:57 329:156 330:253 331:253 332:116 345:40 346:253 347:253 348:253 349:253 350:178 351:27 357:156 358:253 359:253 360:116 372:136 373:204 374:253 375:253 376:253 377:192 378:27 385:156 386:253 387:253 388:116 399:28 400:195 401:254 402:254 403:254 404:250 405:135 412:99 413:255 414:254 415:254 416:117 427:118 428:253 429:253 430:253 431:253 432:142 439:19 440:170 441:253 442:253 443:216 444:62 454:42 455:212 456:253 457:253 458:253 459:253 460:38 466:124 467:188 468:253 469:253 470:253 471:174 482:59 483:253 484:253 485:253 486:237 487:93 488:3 491:31 492:40 493:130 494:247 495:253 496:253 497:253 498:204 499:13 510:59 511:253 512:253 513:253 514:154 518:54 519:218 520:254 521:253 522:253 523:253 524:253 525:253 526:38 538:59 539:253 540:253 541:253 542:215 543:156 544:156 545:156 546:209 547:253 548:255 549:253 550:253 551:253 552:192 553:97 554:15 566:55 567:242 568:253 569:253 570:253 571:253 572:253 573:253 574:253 575:253 576:254 577:253 578:253 579:204 580:23 595:118 596:253 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:254 605:216 606:174 607:13 623:54 624:116 625:243 626:253 627:253 628:253 629:253 630:253 631:146 632:117 633:62 653:53 654:132 655:253 656:253 657:192 658:57 659:13 +1 125:42 126:232 127:254 128:58 153:86 154:253 155:253 156:58 181:86 182:253 183:253 184:58 209:206 210:253 211:253 212:58 237:215 238:253 239:253 240:58 265:215 266:253 267:253 268:58 293:215 294:253 295:253 296:58 321:215 322:253 323:253 324:58 349:215 350:253 351:253 352:58 377:215 378:253 379:253 380:58 405:215 406:253 407:253 408:58 433:188 434:253 435:253 436:85 461:86 462:253 463:253 464:200 465:12 489:29 490:223 491:253 492:253 493:151 518:209 519:253 520:253 521:194 546:128 547:253 548:253 549:200 550:8 574:32 575:213 576:253 577:253 578:152 579:6 603:32 604:221 605:253 606:253 607:153 608:5 632:90 633:215 634:253 635:253 636:151 661:59 662:253 663:253 664:84 +1 156:60 157:229 158:38 184:187 185:254 186:78 211:121 212:252 213:254 214:78 239:197 240:254 241:206 242:6 267:197 268:254 269:202 294:27 295:218 296:233 297:62 322:117 323:254 324:195 350:203 351:254 352:195 377:64 378:244 379:254 380:195 405:79 406:254 407:255 408:161 433:79 434:254 435:254 436:65 461:79 462:254 463:241 464:52 489:79 490:254 491:189 517:79 518:254 519:189 545:79 546:254 547:189 573:79 574:254 575:189 601:79 602:254 603:194 604:5 629:35 630:219 631:254 632:72 658:34 659:223 660:195 687:129 688:195 +1 101:11 102:150 103:72 129:37 130:251 131:71 157:63 158:251 159:71 185:217 186:251 187:71 213:217 214:251 215:71 240:145 241:253 242:253 243:72 267:42 268:206 269:251 270:251 271:71 295:99 296:251 297:251 298:251 299:71 323:253 324:251 325:251 326:251 327:71 350:130 351:253 352:251 353:251 354:251 355:71 377:110 378:253 379:255 380:253 381:253 382:253 383:72 405:109 406:251 407:253 408:251 409:251 410:188 411:30 433:109 434:251 435:253 436:251 437:246 438:123 460:16 461:170 462:251 463:253 464:251 465:215 488:37 489:251 490:251 491:253 492:251 493:86 516:218 517:253 518:253 519:255 520:253 521:35 543:84 544:236 545:251 546:251 547:253 548:168 549:15 571:144 572:251 573:251 574:251 575:190 576:15 599:144 600:251 601:251 602:251 603:180 626:53 627:221 628:251 629:251 630:251 631:180 +0 127:45 128:254 129:254 130:254 131:148 132:24 133:9 154:43 155:254 156:252 157:252 158:252 159:252 160:252 161:121 162:13 181:58 182:237 183:254 184:252 185:252 186:252 187:252 188:252 189:252 190:68 208:69 209:224 210:252 211:254 212:252 213:252 214:252 215:252 216:252 217:252 218:135 219:17 235:75 236:216 237:252 238:252 239:254 240:231 241:168 242:252 243:252 244:252 245:252 246:252 247:45 262:77 263:212 264:252 265:252 266:252 267:242 268:93 269:32 270:114 271:177 272:252 273:252 274:252 275:158 276:12 289:75 290:212 291:252 292:252 293:252 294:252 295:231 299:116 300:252 301:252 302:252 303:252 304:21 316:69 317:216 318:252 319:252 320:252 321:252 322:252 323:62 327:116 328:252 329:252 330:252 331:252 332:21 344:93 345:252 346:252 347:252 348:252 349:252 350:62 355:21 356:158 357:252 358:252 359:252 360:21 371:64 372:239 373:252 374:252 375:252 376:252 377:252 378:21 384:139 385:252 386:252 387:252 388:21 398:5 399:87 400:254 401:254 402:254 403:254 404:237 405:41 411:11 412:150 413:254 414:254 415:254 416:22 425:5 426:85 427:252 428:252 429:252 430:252 431:222 432:55 439:116 440:252 441:252 442:252 443:214 444:18 453:24 454:252 455:252 456:252 457:252 458:252 459:91 466:26 467:153 468:252 469:252 470:252 471:45 481:24 482:252 483:252 484:252 485:252 486:252 487:91 492:18 493:93 494:151 495:252 496:252 497:252 498:184 499:28 509:24 510:252 511:252 512:252 513:252 514:252 515:164 516:116 517:116 518:116 519:117 520:141 521:252 522:252 523:252 524:252 525:252 526:68 537:24 538:252 539:252 540:252 541:252 542:252 543:252 544:252 545:252 546:252 547:254 548:252 549:252 550:252 551:252 552:252 553:163 554:31 565:9 566:121 567:252 568:252 569:252 570:252 571:252 572:252 573:252 574:252 575:254 576:252 577:252 578:252 579:178 580:91 581:33 594:13 595:119 596:252 597:252 598:252 599:252 600:252 601:252 602:252 603:254 604:252 605:252 606:184 607:37 623:13 624:121 625:252 626:252 627:252 628:252 629:252 630:252 631:254 632:214 633:45 634:28 652:8 653:21 654:21 655:169 656:252 657:252 658:41 659:22 660:18 +0 125:218 126:253 127:253 128:255 129:149 130:62 151:42 152:144 153:236 154:251 155:251 156:253 157:251 158:236 159:144 160:144 179:99 180:251 181:251 182:251 183:225 184:253 185:251 186:251 187:251 188:251 189:166 190:16 206:79 207:253 208:251 209:251 210:204 211:41 212:143 213:205 214:251 215:251 216:251 217:253 218:169 219:15 233:79 234:231 235:253 236:251 237:225 238:41 241:41 242:226 243:251 244:251 245:253 246:251 247:164 260:37 261:253 262:253 263:255 264:253 265:35 271:79 272:232 273:255 274:253 275:227 276:42 288:140 289:251 290:251 291:253 292:168 293:15 300:77 301:253 302:251 303:251 304:142 315:21 316:221 317:251 318:251 319:164 320:15 329:227 330:251 331:251 332:236 333:61 342:32 343:190 344:251 345:251 346:251 357:73 358:251 359:251 360:251 361:71 370:73 371:251 372:251 373:251 374:251 385:73 386:251 387:251 388:251 389:71 398:73 399:253 400:253 401:253 402:201 413:73 414:253 415:253 416:253 417:72 426:176 427:251 428:251 429:251 430:71 441:73 442:251 443:251 444:251 445:71 454:253 455:251 456:251 457:157 458:10 469:73 470:251 471:251 472:251 473:71 482:253 483:251 484:251 485:142 497:150 498:251 499:251 500:204 501:41 510:124 511:251 512:251 513:220 514:180 524:130 525:253 526:251 527:225 528:41 538:73 539:253 540:253 541:253 542:253 543:73 544:73 545:10 549:42 550:73 551:150 552:253 553:255 554:253 555:216 566:31 567:189 568:251 569:251 570:251 571:253 572:251 573:159 574:144 575:144 576:145 577:206 578:251 579:251 580:251 581:253 582:168 583:92 595:20 596:195 597:251 598:251 599:253 600:251 601:251 602:251 603:251 604:253 605:251 606:251 607:251 608:225 609:164 610:15 624:21 625:142 626:220 627:253 628:251 629:251 630:251 631:251 632:253 633:251 634:251 635:204 636:41 654:51 655:72 656:174 657:251 658:251 659:251 660:253 661:147 662:71 663:41 +0 127:60 128:96 129:96 130:48 153:16 154:171 155:228 156:253 157:251 158:220 159:51 160:32 181:127 182:251 183:251 184:253 185:251 186:251 187:251 188:251 189:80 207:24 208:182 209:236 210:251 211:211 212:189 213:236 214:251 215:251 216:251 217:242 218:193 234:100 235:194 236:251 237:251 238:211 239:35 241:71 242:173 243:251 244:251 245:253 246:240 247:158 248:19 261:64 262:253 263:255 264:253 265:205 266:19 271:40 272:218 273:255 274:253 275:253 276:91 288:16 289:186 290:251 291:253 292:247 293:110 300:39 301:233 302:251 303:251 304:188 315:16 316:189 317:251 318:251 319:205 320:110 329:48 330:220 331:251 332:220 333:48 343:72 344:251 345:251 346:251 347:158 358:51 359:251 360:251 361:232 371:190 372:251 373:251 374:251 375:59 386:32 387:251 388:251 389:251 398:96 399:253 400:253 401:253 402:95 414:32 415:253 416:253 417:193 426:214 427:251 428:251 429:204 430:23 442:52 443:251 444:251 445:94 454:253 455:251 456:251 457:109 469:48 470:221 471:251 472:219 473:47 482:253 483:251 484:251 485:70 497:234 498:251 499:251 500:188 510:253 511:251 512:251 513:188 523:40 524:158 525:253 526:251 527:172 528:70 539:191 540:253 541:253 542:253 543:96 544:24 549:12 550:174 551:253 552:253 553:255 554:221 567:71 568:251 569:251 570:251 571:253 572:205 573:190 574:190 575:190 576:191 577:197 578:251 579:251 580:231 581:221 582:93 595:16 596:126 597:236 598:251 599:253 600:251 601:251 602:251 603:251 604:253 605:251 606:251 607:140 608:47 625:67 626:188 627:189 628:188 629:188 630:188 631:188 632:189 633:188 634:109 635:4 +0 126:32 127:202 128:255 129:253 130:253 131:175 132:21 152:84 153:144 154:190 155:251 156:253 157:251 158:251 159:251 160:174 176:6 177:37 178:166 179:218 180:236 181:251 182:251 183:251 184:253 185:251 186:251 187:251 188:251 189:156 204:115 205:251 206:251 207:253 208:251 209:251 210:251 211:251 212:253 213:251 214:251 215:251 216:251 217:180 231:105 232:241 233:251 234:251 235:253 236:251 237:251 238:251 239:122 240:72 241:71 242:71 243:148 244:251 245:180 258:73 259:253 260:253 261:253 262:253 263:202 264:253 265:253 266:143 286:31 287:189 288:251 289:251 290:251 291:31 292:189 293:251 294:142 314:63 315:236 316:251 317:251 318:96 320:124 321:246 322:142 330:21 331:166 332:21 342:73 343:251 344:251 345:251 346:71 349:217 350:142 357:32 358:190 359:251 360:142 370:73 371:251 372:251 373:251 374:71 377:217 378:142 385:73 386:251 387:251 388:142 398:73 399:253 400:253 401:253 402:72 405:156 406:103 413:73 414:253 415:253 416:253 417:72 426:73 427:251 428:251 429:251 430:174 441:73 442:251 443:251 444:251 445:71 454:73 455:251 456:251 457:251 458:251 469:73 470:251 471:251 472:251 473:71 482:42 483:205 484:251 485:251 486:251 487:79 497:73 498:251 499:251 500:251 501:71 511:41 512:226 513:251 514:251 515:232 516:77 525:73 526:251 527:251 528:251 529:71 540:166 541:253 542:253 543:255 544:253 545:227 546:73 547:21 553:125 554:253 555:253 556:143 568:16 569:169 570:251 571:253 572:251 573:251 574:251 575:174 576:105 579:63 580:144 581:253 582:251 583:251 584:142 597:15 598:35 599:253 600:251 601:251 602:251 603:251 604:243 605:217 606:217 607:231 608:251 609:253 610:251 611:220 612:20 627:143 628:142 629:236 630:251 631:251 632:253 633:251 634:251 635:251 636:251 637:253 638:251 639:137 657:61 658:71 659:200 660:253 661:251 662:251 663:251 664:251 665:201 666:71 667:10 +1 130:218 131:170 132:108 157:32 158:227 159:252 160:232 185:129 186:252 187:252 188:252 212:1 213:253 214:252 215:252 216:168 240:144 241:253 242:252 243:236 244:62 268:144 269:253 270:252 271:215 296:144 297:253 298:252 299:112 323:21 324:206 325:253 326:252 327:71 351:99 352:253 353:255 354:119 378:63 379:242 380:252 381:253 382:35 406:94 407:252 408:252 409:154 410:10 433:145 434:237 435:252 436:252 461:255 462:253 463:253 464:108 487:11 488:155 489:253 490:252 491:179 492:15 514:11 515:150 516:252 517:253 518:200 519:20 542:73 543:252 544:252 545:253 546:97 569:47 570:233 571:253 572:253 596:1 597:149 598:252 599:252 600:252 624:1 625:252 626:252 627:246 628:132 652:1 653:169 654:252 655:132 +1 130:116 131:255 132:123 157:29 158:213 159:253 160:122 185:189 186:253 187:253 188:122 213:189 214:253 215:253 216:122 241:189 242:253 243:253 244:122 267:2 268:114 269:243 270:253 271:186 272:19 295:100 296:253 297:253 298:253 299:48 323:172 324:253 325:253 326:253 327:48 351:172 352:253 353:253 354:182 355:19 378:133 379:251 380:253 381:175 382:4 405:107 406:251 407:253 408:253 409:65 432:26 433:194 434:253 435:253 436:214 437:40 459:105 460:205 461:253 462:253 463:125 464:40 487:139 488:253 489:253 490:253 491:81 514:41 515:231 516:253 517:253 518:159 519:16 541:65 542:155 543:253 544:253 545:172 546:4 569:124 570:253 571:253 572:253 573:98 597:124 598:253 599:253 600:214 601:41 624:22 625:207 626:253 627:253 628:139 653:124 654:253 655:162 656:9 diff --git a/data/mllib/sample_linear_regression_data.txt b/data/mllib/sample_linear_regression_data.txt new file mode 100755 index 0000000000000..9aaaa4297548b --- /dev/null +++ b/data/mllib/sample_linear_regression_data.txt @@ -0,0 +1,501 @@ +-9.490009878824548 1:0.4551273600657362 2:0.36644694351969087 3:-0.38256108933468047 4:-0.4458430198517267 5:0.33109790358914726 6:0.8067445293443565 7:-0.2624341731773887 8:-0.44850386111659524 9:-0.07269284838169332 10:0.5658035575800715 +0.2577820163584905 1:0.8386555657374337 2:-0.1270180511534269 3:0.499812362510895 4:-0.22686625128130267 5:-0.6452430441812433 6:0.18869982177936828 7:-0.5804648622673358 8:0.651931743775642 9:-0.6555641246242951 10:0.17485476357259122 +-4.438869807456516 1:0.5025608135349202 2:0.14208069682973434 3:0.16004976900412138 4:0.505019897181302 5:-0.9371635223468384 6:-0.2841601610457427 7:0.6355938616712786 8:-0.1646249064941625 9:0.9480713629917628 10:0.42681251564645817 +-19.782762789614537 1:-0.0388509668871313 2:-0.4166870051763918 3:0.8997202693189332 4:0.6409836467726933 5:0.273289095712564 6:-0.26175701211620517 7:-0.2794902492677298 8:-0.1306778297187794 9:-0.08536581111046115 10:-0.05462315824828923 +-7.966593841555266 1:-0.06195495876886281 2:0.6546448480299902 3:-0.6979368909424835 4:0.6677324708883314 5:-0.07938725467767771 6:-0.43885601665437957 7:-0.608071585153688 8:-0.6414531182501653 9:0.7313735926547045 10:-0.026818676347611925 +-7.896274316726144 1:-0.15805658673794265 2:0.26573958270655806 3:0.3997172901343442 4:-0.3693430998846541 5:0.14324061105995334 6:-0.25797542063247825 7:0.7436291919296774 8:0.6114618853239959 9:0.2324273700703574 10:-0.25128128782199144 +-8.464803554195287 1:0.39449745853945895 2:0.817229160415142 3:-0.6077058562362969 4:0.6182496334554788 5:0.2558665508269453 6:-0.07320145794330979 7:-0.38884168866510227 8:0.07981886851873865 9:0.27022202891277614 10:-0.7474843534024693 +2.1214592666251364 1:-0.005346215048158909 2:-0.9453716674280683 3:-0.9270309666195007 4:-0.032312290091389695 5:0.31010676221964206 6:-0.20846743965751569 7:0.8803449313707621 8:-0.23077831216541722 9:0.29246395759528565 10:0.5409312755478819 +1.0720117616524107 1:0.7880855916368177 2:0.19767407429003536 3:0.9520689432368168 4:-0.845829774129496 5:0.5502413918543512 6:-0.44235539500246457 7:0.7984106594591154 8:-0.2523277127589152 9:-0.1373808897290778 10:-0.3353514432305029 +-13.772441561702871 1:-0.3697050572653644 2:-0.11452811582755928 3:-0.807098168238352 4:0.4903066124307711 5:-0.6582805242342049 6:0.6107814398427647 7:-0.7204208094262783 8:-0.8141063661170889 9:-0.9459402662357332 10:0.09666938346350307 +-5.082010756207233 1:-0.43560342773870375 2:0.9349906440170221 3:0.8090021580031235 4:-0.3121157071110545 5:-0.9718883630945336 6:0.6191882496201251 7:0.0429886073795116 8:0.670311110015402 9:0.16692329718223786 10:0.37649213869502973 +7.887786536531237 1:0.11276440263810383 2:-0.7684997525607482 3:0.1770172737885798 4:0.7902845707138706 5:0.2529503304079441 6:-0.23483801763662826 7:0.8072501895004851 8:0.6673992021927047 9:-0.4796127376677324 10:0.9244724404994455 +14.323146365332388 1:-0.2049276879687938 2:0.1470694373531216 3:-0.48366999792166787 4:0.643491115907358 5:0.3183669486383729 6:0.22821350958477082 7:-0.023605251086149304 8:-0.2770587742156372 9:0.47596326458377436 10:0.7107229819632654 +-20.057482615789212 1:-0.3205057828114841 2:0.51605972926996 3:0.45215640988181516 4:0.01712446974606241 5:0.5508198371849293 6:-0.2478254241316491 7:0.7256483175955235 8:0.39418662792516 9:-0.6797384914236382 10:0.6001217520150142 +-0.8995693247765151 1:0.4508991072414843 2:0.589749448443134 3:0.6464818311502738 4:0.7005669004769028 5:0.9699584106930381 6:-0.7417466269908464 7:0.22818964839784495 8:0.08574936236270037 9:-0.6945765138377225 10:0.06915201979238828 +-19.16829262296376 1:0.09798746565879424 2:-0.34288007110901964 3:0.440249350802451 4:-0.22440768392359534 5:-0.9695067570891225 6:-0.7942032659310758 7:-0.792286205517398 8:-0.6535487038528798 9:0.7952676470618951 10:-0.1622831617066689 +5.601801561245534 1:0.6949189734965766 2:-0.32697929564739403 3:-0.15359663581829275 4:-0.8951865090520432 5:0.2057889391931318 6:-0.6676656789571533 7:-0.03553655732400762 8:0.14550349954571096 9:0.034600542078191854 10:0.4223352065067103 +-3.2256352187273354 1:0.35278245969741096 2:0.7022211035026023 3:0.5686638754605697 4:-0.4202155290448111 5:-0.26102723928249216 6:0.010688215941416779 7:-0.4311544807877927 8:0.9500151672991208 9:0.14380635780710693 10:-0.7549354840975826 +1.5299675726687754 1:-0.13079299081883855 2:0.0983382230287082 3:0.15347083875928424 4:0.45507300685816965 5:0.1921083467305864 6:0.6361110540492223 7:0.7675261182370992 8:-0.2543488202081907 9:0.2927051050236915 10:0.680182444769418 +-0.250102447941961 1:-0.8062832278617296 2:0.8266289890474885 3:0.22684501241708888 4:0.1726291966578266 5:-0.6778773666126594 6:0.9993906921393696 7:0.1789490173139363 8:0.5584053824232391 9:0.03495894704368174 10:-0.8505720014852347 +12.792267926563595 1:-0.008461200645088818 2:-0.648273596036564 3:-0.005334477339629995 4:0.3781469006858833 5:0.30565234666790686 6:-0.2822867492866177 7:0.10175120738413801 8:0.5342432888482425 9:0.05146513075475534 10:-0.6459729964194652 +6.082192787194888 1:0.42519013450094767 2:0.09441503345243984 3:-0.07898439043103522 4:-0.32207498048636474 5:-0.9180071861219266 6:0.5951317320731633 7:0.41000814588717693 8:-0.3926260640533046 9:0.2789036768568971 10:0.13163692286014528 +-7.481405271455238 1:0.03324842612749346 2:0.07055844751995122 3:-0.47199515597021113 4:-0.682690342465275 5:0.3983414713797069 6:-0.2136729393256811 7:-0.09066563475481249 8:-0.4640338194317184 9:-0.03513782089224482 10:-0.1711809802758364 +6.739533816100517 1:0.1774546460228057 2:-0.6783644553523549 3:-0.47871398278230504 4:0.02272121490463097 5:-0.5047649289302389 6:0.26479596144873896 7:-0.32045436544054096 8:0.3113047940487379 9:0.6269418147567556 10:0.9710114516962312 +3.780807062175497 1:0.01715676997104909 2:0.8975962429865936 3:-0.46594560920034134 4:0.2873623499953055 5:0.8894362304584083 6:0.17973981232418468 7:0.49105791400707743 8:-0.7359842740294882 9:0.38941133808001127 10:-0.7151884777228046 +4.564039393483412 1:0.07478785545033317 2:-0.8672651994084235 3:0.450599300176334 4:0.35104802298560056 5:0.6797318185095045 6:-0.03891997518827006 7:-0.33208695871398675 8:0.6166574577055226 9:0.5730212324012205 10:-0.4194925751047054 +-0.3195679646035633 1:0.054527683864544096 2:-0.15591931640565093 3:0.9266742559542833 4:0.888522581905147 5:0.6576203900699167 6:0.6417770212400336 7:0.7509788029052338 8:-0.3104974571382815 9:0.7234744267051683 10:-0.15869049651427103 +11.290452658023497 1:0.20173310976772196 2:0.8657502566551409 3:0.9325160601080682 4:0.24570884032596263 5:-0.6546108813337841 6:-0.14020032028377583 7:-0.8825687891702743 8:-0.21420166926412865 9:-0.8600275184792756 10:-0.7990574622230739 +-4.003499192090455 1:0.8325875503351796 2:-0.5956350140619129 3:0.12598048009007923 4:0.12340188733473134 5:-0.839435659309717 6:-0.16623481818728414 7:0.12028795301041662 8:-0.7994713170657952 9:0.2216721974907896 10:0.8407561415075087 +-19.872991038068406 1:-0.9325810772922609 2:-0.6411471147334535 3:0.9949216290375054 4:0.483048267470493 5:-0.8736297429070232 6:-0.36222771685582544 7:0.26397860162786957 8:0.45527588775737704 9:-0.9424989711186325 10:0.6251162293059616 +10.502762149373098 1:-0.2307778924009991 2:0.6977871128979924 3:0.022830408261390822 4:0.6257738824362347 5:0.9770979848265122 6:0.09985730624684575 7:-0.9755858424230182 8:-0.689969833240031 9:-0.7294587311376761 10:0.3496326193951331 +-14.328978509075442 1:0.37929821892417404 2:0.8402056881660709 3:-0.1806835799958202 4:0.766314307210441 5:0.865876513623024 6:-0.7113501219432434 7:-0.0932956557986735 8:-0.7042025810921411 9:0.47530696925672267 10:-0.4629102077669889 +-16.26143027545273 1:-0.9309578475799722 2:0.7591795880911123 3:0.06296957473213705 4:0.786790093290086 5:-0.9527998391625465 6:-0.08573982501921895 7:-0.3812232026687308 8:-0.6890669703685022 9:0.25415911467755015 10:-0.07664746267502509 +11.772544195529013 1:0.3614756404325046 2:0.14508027508253818 3:0.23042774014795753 4:0.4164348685332022 5:0.4109091750657461 6:0.03853098236933272 7:0.38911994885223145 8:-0.5031309357181766 9:-0.596467768575587 10:0.17884522225228028 +14.697703557439503 1:0.24508864174863 2:0.7576193329655578 3:0.09030511120334461 4:0.9537528991778741 5:-0.7224092160621338 6:-0.34089385162121943 7:0.6924170720838818 8:0.32912306214891784 9:-0.4064624712125904 10:-0.5344662061201593 +-13.976130931152703 1:0.5891192531479754 2:0.29862103742464274 3:-0.36153976712796343 4:-0.6552669564323226 5:-0.22672513691161766 6:0.3001336202535376 7:0.34490251346382617 8:0.2072633053920192 9:-0.5659371284058774 10:0.49599636156628835 +-14.762758252931127 1:0.31302496164254223 2:-0.6062773982342133 3:-0.9874007658402217 4:-0.6214904627601421 5:-0.11421073677207683 6:-0.5850843421161205 7:0.1250679146774638 8:-0.7108170726393621 9:-0.6888351241194393 10:0.6077343683084389 +-3.300641320608255 1:-0.1407178879203672 2:0.12960233233004925 3:-0.4236196478321872 4:0.7903078296084356 5:-0.8755754953628643 6:-0.2062360260394529 7:-0.045680124889026175 8:0.783182093429277 9:-0.02995737262668463 10:-0.33382351650328435 +-15.72351561304857 1:-0.1802575775708093 2:-0.991006951265341 3:-0.9107951763247621 4:0.9069820084047908 5:-0.12691921206803047 6:-0.7087012119383593 7:-0.9179510577925369 8:0.18480349982718325 9:-0.4478459144114004 10:-0.5560585660624608 +-22.949825936196074 1:0.4797855980916854 2:0.01997502546020402 3:-0.8827928315487465 4:0.2755107907750989 5:0.015544482147298977 6:0.9652687138748801 7:0.6622667860970648 8:-0.7708138539912186 9:0.17728148663006627 10:0.47818190728952925 +12.092431628826905 1:0.1358843437335564 2:0.03643446587894239 3:-0.31070823939673287 4:0.5283033206569152 5:0.3469111543845367 6:-0.5162518174930761 7:0.24270234207184016 8:0.7352292800096338 9:0.8860322286740037 10:0.6748068653962045 +-23.51088409032297 1:-0.4683538422180036 2:0.1469540185936138 3:0.9113612952591796 4:-0.9838482669789823 5:0.4506466371133697 6:0.6456121712599778 7:0.8264783725578371 8:0.562664168655115 9:-0.8299281852090683 10:0.40690300256653256 +5.998186124881712 1:-0.9781302074883151 2:0.32984303335155785 3:0.7303430847899663 4:0.841481297188956 5:0.05580773881989276 6:0.7130788298702062 7:-0.218087116119847 8:-0.9889494995220598 9:0.9182854134226501 10:-0.7501751701020942 +9.852316338642547 1:0.146854160091757 2:-0.3611508707370965 3:0.3517016971654914 4:0.6187697988029395 5:-0.010768583697787548 6:0.5236725885871243 7:0.5945666964145524 8:-0.009180562740628506 9:-0.44474762415618274 10:0.41852743519493685 +-5.313930756588526 1:-0.6304209277071555 2:-0.37010359785263813 3:-0.3194739026510125 4:-0.750533359080716 5:0.45500303301733114 6:-0.012727544364283805 7:-0.43941651856862274 8:0.927108876532093 9:-0.24164903158058149 10:0.44134972919002124 +-4.2775224863223915 1:-0.35785764991284363 2:0.942797043714243 3:0.4539569191274251 4:-0.6944903010994341 5:-0.08357221983075225 6:0.4433049548665855 7:-0.5488972050023557 8:-0.24014623658145773 9:-0.6178118485382511 10:-0.4575463952834564 +-10.57769830424322 1:0.22693864400257335 2:-0.041639691095668674 3:0.9948726461115123 4:-0.7450471554938383 5:-0.1114847126717804 6:-0.27881184842402673 7:0.029766812446276214 8:-0.3727649352432578 9:-0.7791732805568077 10:0.9425576681069683 +-0.8430338600258201 1:0.4607090007225536 2:-0.6079961642969514 3:-0.5671626932935381 4:0.12784576080614185 5:-0.30766031989910236 6:-0.21232963505711555 7:0.3310463755850872 8:-0.6807682731528943 9:0.7826634145951483 10:0.0608057623636995 +13.450586257053727 1:-0.2697769964284986 2:0.07743737732312428 3:-0.8459687499864881 4:0.6091901514177853 5:-0.9464815428211699 6:0.15780407422581533 7:-0.28552052619478996 8:-0.27500859181806403 9:-0.7207541548282903 10:0.05215593729084533 +20.358241877831016 1:0.29768927445620164 2:-0.5379390525163252 3:0.6591913001003027 4:0.6635992348010928 5:0.3786594651413009 6:-0.7217135278882543 7:0.9634013908615768 8:0.03961253903778861 9:0.1335121312144949 10:0.7933944303463509 +9.800993960518852 1:0.39896823489212285 2:0.30948413101894023 3:0.08568060094378493 4:-0.7454513450113371 5:0.8054125831421357 6:-0.24464240413169347 7:-0.18294406588625112 8:-0.883455504399858 9:0.2468431033653562 10:-0.708151566382103 +-21.432387764165806 1:-0.4785033857256795 2:0.520350718059089 3:-0.2988515012130126 4:-0.46260150057299754 5:0.5394344995663083 6:0.39320468081626836 7:0.1890560923345248 8:0.13123799325264507 9:0.43613839380760355 10:0.39541998419731494 +-4.090570760187878 1:0.3909705814857716 2:0.9830271975811611 3:0.672523651785939 4:0.0035177223850744177 5:0.567082732451311 6:-0.2620454326881394 7:0.46622578556708105 8:0.646246879249865 9:0.4263175536668733 10:0.8982696975276223 +3.7459201216906926 1:-0.9480167656870653 2:-4.888270196095057E-4 3:0.48226844071577646 4:-0.23706663537631645 5:0.22420266627462127 6:0.2981747607694978 7:0.3893425967975348 8:0.6302701381298614 9:-0.21909113816064196 10:0.8371697958140494 +9.767952084958061 1:-0.2300790371078303 2:-0.4457883630748676 3:0.28710853302295325 4:0.7112839743052013 5:-0.8765858382640623 6:-0.6470779468607217 7:0.4369262584371727 8:-0.7175412028407337 9:0.5506733477278882 10:0.5393007189573547 +6.9802839308913365 1:0.21769855012808215 2:0.8653818331675485 3:0.2322943113578111 4:0.3760591265797468 5:0.06554014167292377 6:0.6866096712933549 7:0.866929973115441 8:-0.6462263417217329 9:0.2507247465275353 10:-0.7005877782050307 +16.014720800069103 1:0.6058055248984549 2:0.048517868234337014 3:-0.15744912875924877 4:0.32598079708869365 5:-0.587791997223768 6:-0.4636187312118474 7:0.7771908559246068 8:-0.349403853888719 9:0.229800030145503 10:-0.674614818934488 +8.417571532985823 1:-0.21164946152466801 2:-0.9981936663594053 3:0.8611869575187896 4:0.11100891297254312 5:-0.7406067304729631 6:-0.7613837395522254 7:-0.9617573325708704 8:0.5697426971647488 9:-0.5830879716990833 10:0.5951448538064159 +-12.491442077546413 1:-0.19172117564625735 2:-0.12421304883392126 3:0.7095605786791346 4:0.6401582292398038 5:-0.9347790209840108 6:0.6592209285686903 7:0.702282297844389 8:-0.22765902007749528 9:-0.17746922342943816 10:0.7196663432778121 +-8.605713514762092 1:0.36490454976480846 2:0.6991204480538957 3:0.6546945560337121 4:-0.032324845758738174 5:0.2453935969836043 6:0.5363119225093116 7:0.6266741350524205 8:-0.2132266305382322 9:-0.308105870487996 10:-0.08219413867616465 +-10.35591860037468 1:-0.014204168485027147 2:-0.7077035677144325 3:0.024004217785642767 4:0.818971992516166 5:0.9081305263471056 6:0.808854493237229 7:-0.6474336785461867 8:-0.32559288177031465 9:-0.32850453072496055 10:-0.7035310416695784 +3.605002621628445 1:0.6085817977516599 2:0.8101072412357928 3:0.7697891508923966 4:-0.5738750389864677 5:-0.734314989863889 6:-0.7879014492215499 7:0.6884442838920775 8:-0.46131231930402383 9:-0.7730585954271005 10:-0.7819874019145132 +12.30435312415091 1:0.3283668768730639 2:-0.18316686990068187 3:0.3955614099142126 4:0.8450470350842108 5:0.3958042901611589 6:0.6578475571960676 7:-0.4395488558075096 8:0.15720430113495376 9:-0.5318362828977672 10:0.45585285255232044 +9.020048819638827 1:-0.5986521145193395 2:0.3266542215286443 3:-0.09911773729611917 4:-0.21478254478908676 5:0.6546175049764293 6:-0.1414796368932345 7:0.25802631337510085 8:-0.6773828562539816 9:-0.22038193899258718 10:-0.17465737306657902 +14.854262978981406 1:0.5293763924477841 2:-0.24658868331583683 3:0.8268631648872109 4:0.8969207203400265 5:0.03933229861213983 6:-0.6212951181360529 7:-0.36695460282178205 8:-0.5468014636386027 9:-0.3419492829414976 10:-0.8273314086998671 +5.658665647926016 1:0.9543096383762801 2:0.13230023957687176 3:-0.3071929861496465 4:-0.3646067841449696 5:0.6979929890816723 6:-0.20721664168809228 7:0.6676482547655365 8:0.944757051233543 9:0.024377296173674567 10:-0.9413728609667691 +-6.930603551528371 1:0.09198647857985232 2:-0.3685113649452161 3:-0.2361728930325453 4:0.3674268130607439 5:0.27385598384498344 6:-0.7151900241735676 7:0.3310154476154119 8:-0.24328111897361682 9:0.2511378679668912 10:-0.35825141175578934 +13.361196783041926 1:0.11676665169094824 2:-0.49968608916548307 3:0.9941342810313298 4:-0.17858967215374988 5:0.1993744673440312 6:0.14596837574280297 7:-0.8245495433125194 8:-0.5637934691545672 9:-0.8589185806222286 10:-0.4923216901915597 +-3.280508467210429 1:-0.9917770074538397 2:-0.1547651813493751 3:0.621733177563484 4:0.7303326279246298 5:-0.0786900332560696 6:0.9107127797641994 7:0.7104513024299466 8:-0.32858522942354407 9:0.17013652749847386 10:0.27656984316288824 +11.13509519160867 1:0.6874932143640391 2:-0.46610293161038907 3:0.8744681017967024 4:0.40900365224695956 5:-0.49770054448432055 6:-0.0635770754462921 7:-0.5705387648707747 8:-0.577988250149829 9:-0.8099463063934682 10:0.42132700180827354 +-11.857350365429426 1:-0.24607974991258308 2:-0.943388538022258 3:0.8679112109377674 4:0.7779951176637694 5:-0.5802336023276593 6:-0.9093352471884992 7:0.29337797938742316 8:0.498519874589175 9:0.3493034812120912 10:-0.07307210651399076 +11.421632138263703 1:0.3911519359353859 2:-0.8154393787235621 3:0.47194271125243237 4:0.14014792298759593 5:-0.3589345913619957 6:0.7887695409762479 7:0.49962792312858895 8:-0.6402670146359797 9:-0.2314041601683119 10:-0.798901341175887 +5.194792012146463 1:0.810279303469398 2:-0.9772756877199589 3:-0.20925958437085557 4:0.8797562461102444 5:0.3211532423260066 6:0.25250279470783754 7:0.14387831263435813 8:-0.021466789385169882 9:0.18909293657271564 10:-0.5981349964027893 +12.242677118499806 1:0.3565715672082048 2:0.7366743237221687 3:0.1922233582434527 4:-0.3551925780624561 5:0.5290849503909634 6:0.7744214641246749 7:0.7277215028580597 8:-0.590440215391044 9:0.7427328184290733 10:-0.6231904162251609 +3.496172341296411 1:0.5028717258135624 2:-0.5838871888624848 3:-0.5540116561110324 4:0.8502487679795261 5:-0.7983061034328727 6:-0.3853123296389005 7:-0.1493800684643869 8:0.6008798629354264 9:-0.32299062155495406 10:-0.5827019502242026 +-15.437384793431217 1:0.41994681418237345 2:0.7106426870657483 3:-0.45211033467567696 4:-0.7272406549392239 5:-0.35736594496490737 6:0.4764507578985955 7:-0.5249912641281373 8:0.8562010912051132 9:0.45927621623833637 10:-0.3701817429794385 +5.490036861541498 1:0.8414999442459015 2:0.9273442862476728 3:-0.054654787893199774 4:-0.23126134156257327 5:-0.9155048245317694 6:0.25750538376376975 7:-0.8470916763665326 8:0.9105674676753848 9:0.5026028522378054 10:-0.06650501561108468 +-1.074065343287859 1:0.37484830603001607 2:-0.9858854245832975 3:0.007159356555897611 4:0.8172796295244154 5:0.519147377529164 6:0.8211049991970722 7:0.9901658817979146 8:-0.026519560032641998 9:-0.2328762488733862 10:0.43161994187258035 +2.0482082496444622 1:0.24940246021565793 2:0.47248358864259177 3:0.23833814894291105 4:-0.3449172512379757 5:0.7412869866239866 6:0.1351422898741914 7:-0.003784141556894216 8:-0.6321917152754075 9:0.8246267827865776 10:0.5057520480449009 +16.709794859608397 1:-0.5977424405191092 2:-0.13991362149785713 3:0.613487896720806 4:-0.37681525320218157 5:-0.4369592282569783 6:0.4702242879506955 7:0.07498463532645339 8:-0.9942304127133292 9:0.41304209196175257 10:0.6799250665519481 +4.598881854940949 1:-0.41212838137243835 2:0.6737124633791323 3:0.8376369191216593 4:0.2848328781926128 5:-0.17960265353296 6:0.0035488712665472377 7:-0.8355355482928055 8:-0.7439716673142398 9:-0.009043467128117433 10:0.7423272515054122 +9.566038608555402 1:-0.662329643040616 2:0.4727113884417973 3:-0.15734218732411365 4:-0.3950754785173889 5:0.13837083076070011 6:0.633261314089351 7:0.9929998062307679 8:-0.4639028424346423 9:-0.073992579817449 10:0.3413166410117088 +1.629198477883475 1:-0.2875719791707101 2:0.9395753700232541 3:-0.45090801750966314 4:-0.384528069378699 5:-0.35937736478702753 6:0.9597102694501136 7:-0.6898325123180971 8:-0.11436012866371303 9:-0.5330550575952768 10:0.24688769932037258 +-7.374620970147229 1:0.16864051681940984 2:-0.08391828256018252 3:-0.8184503043836224 4:0.5461252511055263 5:0.7264676659099087 6:-0.9654384426822686 7:-0.8537533138667612 8:0.9189716013058653 9:-0.03449322582531389 10:0.5490329745887035 +-0.5741704240890674 1:0.9392753294760656 2:-0.5579682000156501 3:-0.8083270703362093 4:-0.7022804026958895 5:-0.30426803430649896 6:0.8211432527140852 7:-0.8101343265051797 8:-0.0945946325760949 9:0.49546915718101814 10:0.5184327698839013 +12.583032451116004 1:0.20496323995364651 2:0.5082017540304999 3:0.2428646053751764 4:0.7101854338863274 5:-0.9619925264660094 6:0.4610134502825909 7:-0.5620669052678122 8:0.6766614078376236 9:-0.7169693435782278 10:-0.14362322382035164 +-10.489157123372898 1:-0.7441633083637054 2:0.07069898351187809 3:-0.47119552972566336 4:-0.43970155900871344 5:0.43192289605353973 6:-0.0798550143899397 7:0.2111188135787776 8:0.9101748615761336 9:-0.4079984876629721 10:-0.8101424982394589 +-3.811365493249739 1:0.7250263461647963 2:0.22182621035333838 3:-0.12735342714215725 4:0.26222861719040624 5:0.3928174057935714 6:0.817131411734006 7:-0.056109765698795 8:0.7908779197353637 9:-0.06768319505245768 10:0.4107045608924882 +-7.604636483513961 1:0.876751634787073 2:0.04037085575852295 3:0.18142385658771398 4:0.38350565074271903 5:-0.30937664332011905 6:-0.9544807672006823 7:0.008643477632712449 8:-0.27676843472226276 9:-0.12938540988602476 10:-0.2929762262661819 +-1.9889499615051784 1:-0.4243149295090465 2:0.22578711943818686 3:0.662530786460152 4:0.28592235843136105 5:0.4170345231441832 6:0.9290881132120887 7:0.5332443368002588 8:-0.33248958421809927 9:0.16273139830495942 10:0.6899022585936985 +-1.99891354174786 1:-0.1732078452611825 2:0.2361029542296429 3:-0.8455867017505336 4:0.31638672033240867 5:-0.648387667144986 6:-0.7647886103837449 7:0.6910155501192978 8:-0.2665663102538198 9:-0.5980899570876459 10:-0.9165896495676276 +9.74348630903265 1:0.18934450539532244 2:-0.715110505416745 3:-0.453777527810155 4:0.2743741252197758 5:-0.8439310405443103 6:-0.533835190276116 7:-0.5911710854054728 8:0.21026462628920695 9:-0.45849607678093585 10:0.1327074179200407 +20.221961806051706 1:0.624731930687735 2:-0.39914395421723015 3:0.781887900750925 4:0.5442619051596436 5:0.16651193067479153 6:0.9064846121246533 7:-0.3643159594276202 8:-0.5182065337246469 9:-0.6785628247191553 10:0.7111152852903913 +20.456947955410897 1:-0.21923785332346513 2:0.11340668617783778 3:0.7397883986253251 4:-0.11748081084695605 5:0.06314872700777197 6:-0.7124574845946587 7:0.18043581960897104 8:-0.09023925260092103 9:-0.7256417560118238 10:-0.5038088673851804 +12.241006086129564 1:-0.15271598143132215 2:0.9038942665552285 3:-0.6168514099878155 4:-0.12219038322317011 5:0.5402785935596728 6:0.4059744401803913 7:0.258870596734184 8:0.3190881033039108 9:0.2372469007313076 10:0.367188299614863 +3.980473021620311 1:-0.9025895351376971 2:-0.03333947011476446 3:-0.8220776066161464 4:0.449117985679933 5:0.9970519437779266 6:0.27430911004640457 7:0.039081352882204046 8:-0.8621514950929796 9:-0.569587565933642 10:-0.9118346349929578 +-13.420594775890757 1:0.3697979495309094 2:0.07383664120111888 3:0.7199366131785143 4:0.2118625428869032 5:-0.9015976323216077 6:-0.5298395275757712 7:-0.9517419542156635 8:0.39554920787574743 9:-0.3721957439110324 10:-0.4750272836396878 +-1.052659359353786 1:0.02106845330888185 2:0.7571245678782959 3:0.8034228830223251 4:0.32968340513846917 5:-0.6510386482911554 6:0.2710115488605187 7:-0.1319580272290235 8:0.932600992666184 9:0.8260461527035414 10:-0.8507648952138052 +9.813440129324034 1:0.41048687946340134 2:0.9384639988086239 3:0.4569555844323441 4:-0.3084729082645552 5:-0.7299010284877061 6:-0.6925012997779212 7:-0.6798013915257548 8:-0.504368104320321 9:-0.6234398059664716 10:0.8633407902005543 +-2.8942782378157714 1:0.5546381825677706 2:0.7959405841824887 3:0.584699836289184 4:-0.5726371777829862 5:-0.2827976152663936 6:0.138034013875719 7:-0.2935080791661324 8:-0.5323479091625714 9:0.6837641044797451 10:0.5986680812032501 +8.562937733537664 1:0.14753220510180776 2:-0.31591341855048327 3:-0.748545617199091 4:0.3251888821665734 5:0.8228589483149358 6:0.046659706976506676 7:-0.35049927996132624 8:0.2953170004605874 9:-0.6429374177050204 10:0.4624083116836044 +13.413187970975178 1:-0.7229883396779724 2:0.8876940454894067 3:-0.033794226589695775 4:0.46700071356381523 5:0.1599557295166274 6:-0.8944619785248653 7:-0.1258464584151997 8:-0.8797551785991506 9:-0.14408879184669354 10:0.11020655997336015 +-5.491389764900794 1:-0.366507395597937 2:0.630480481240723 3:-0.16600801981741609 4:0.09842042773854076 5:0.30129535029579047 6:0.14102166298628882 7:-0.28131788612036623 8:0.49635295715686234 9:0.0625636989631968 10:-0.41748132718912 +-10.29566593602992 1:-0.7898597726154271 2:-0.05425577320946573 3:0.5992645759265662 4:-0.4716868549309716 5:-0.020137302700854676 6:0.6216515277233232 7:-0.7295510954484412 8:-0.41443875567123967 9:-0.610576632050404 10:-0.9515988311377204 +7.084732852050431 1:0.9990215581592679 2:-0.9868954542412269 3:0.49133473382040704 4:0.7697599878561228 5:-0.34668939907967267 6:0.9777705993519483 7:0.4449043102759509 8:0.9812971199646168 9:0.6666598587737487 10:0.14398842572598514 +0.23715467505851734 1:0.21628799185444336 2:-0.4526390568867018 3:0.6558486691929235 4:0.13730688681492142 5:0.23076986155942736 6:0.7020484017619715 7:-0.12077999528458938 8:0.8306084972447003 9:-0.49337323198621563 10:-0.8270028152572872 +1.1552619549601455 1:-0.48202394020369277 2:-0.6274878708695264 3:-0.27623674153600697 4:-0.5312153415813432 5:-0.030820182786174044 6:-0.5893370965577813 7:0.6666315120904487 8:-0.36482991729570036 9:0.6065771813692735 10:0.05831057330788525 +-0.20433879835596253 1:-0.4702220250018212 2:0.9123705796362889 3:-0.2045657170490376 4:-0.18922063450309534 5:-0.31431213362503163 6:0.4150130060120387 7:0.34016193625941127 8:0.8391374136299805 9:0.6884250315764333 10:-0.7916408854251566 +-9.751622607785082 1:-0.0014232315621649505 2:-0.1284246813729939 3:0.5228953023175369 4:0.9688522449007109 5:-0.7857721219549156 6:-0.7812922263391038 7:-0.5916136652814756 8:0.793988610184206 9:0.7982949061274296 10:-0.592785473963741 +-22.837460416919342 1:-0.17363144173810174 2:-0.3340314573781735 3:0.9351424971322297 4:-0.6430601902397572 5:-0.13363305808148818 6:-0.42446359566938585 7:-0.4093070316761178 8:-0.9302259781839204 9:0.47004365892170585 10:-0.6231289889808045 +-3.6318714209289436 1:-0.8296410705737971 2:-0.6056572341069668 3:-0.2975417404042737 4:0.07134138175064741 5:-0.8966463747179154 6:-0.4112675899658855 7:0.7908013478009401 8:0.407396254566472 9:0.9227769302156879 10:0.12418427404473764 +-3.8909712376010583 1:-0.6552751548581366 2:-0.5641921108932855 3:-0.6340486345063014 4:-0.5441069121131075 5:0.908720622198947 6:-0.026054643814348077 7:0.03706191653058433 8:-0.6672524338819317 9:0.7958274915288801 10:-0.19029619970124023 +-10.600130341909033 1:-0.7457695999520562 2:-0.3739453132549577 3:0.01327423342620393 4:-0.08482897201178563 5:0.84573456086082 6:0.6279927575103963 7:0.014494803555804125 8:0.9420647557771027 9:-0.13484113287285893 10:0.3037405853352888 +-12.094351278535258 1:0.9982796018306028 2:0.8354271779265348 3:0.46284321795736116 4:0.07693347919601745 5:-0.4753440408996932 6:-0.47098252868073787 7:0.4810729184846003 8:-0.6136990339205741 9:-0.6715833036640317 10:-0.6247058955319091 +9.936399360181602 1:0.7330323083522969 2:0.47204204993669197 3:0.3850471475752122 4:0.21483460195167958 5:0.3806220122265147 6:0.6336993433402796 7:-0.47987416364572 8:-0.195509010865196 9:-0.6561820282562041 10:-0.45300480439842894 +-4.706701061062994 1:-0.847895844561626 2:-0.29946646506145114 3:0.25432868082106497 4:0.1352958872054535 5:-0.8803017974303002 6:-0.3675110562764785 7:0.10888496324899721 8:0.9620737605396772 9:-0.031046632561323895 10:-0.09466883461500908 +5.101614991255809 1:-0.5174248135588373 2:0.14394061894828014 3:0.5613709266711013 4:-0.5678634944326011 5:0.930216209978763 6:-0.6204727890080077 7:0.4133141749872311 8:0.6262685035917408 9:0.03382924477926896 10:-0.15231139191832854 +-8.772667465932606 1:-0.2117605577769197 2:-0.4283897136887762 3:0.44686767473401035 4:-0.5507826261358746 5:0.237124956028401 6:0.6413157520982717 7:0.2409214827604571 8:-0.8505503638033114 9:-0.9811997368468401 10:-0.9499963936664035 +-11.615775265015627 1:0.8782018665273386 2:-0.9751473570197167 3:0.6307050068521085 4:0.7012721336851997 5:0.21318736263512283 6:0.024885128053773853 7:-0.4580644243558505 8:0.1318650007251434 9:-0.9306090092992167 10:-0.5688746770986652 +19.64829023536192 1:0.14426537998360645 2:0.3557716894181753 3:-0.8577143134654568 4:0.5288643233801469 5:0.9231529738221469 6:0.975999712077738 7:0.24700404691888678 8:0.10206517527052283 9:-0.10041951294847062 10:-0.9412918491876225 +2.7409415438025486 1:-0.7404936009304737 2:-0.9792071376296605 3:-0.49510748520932113 4:0.9538460112904268 5:-0.5075114153141447 6:-0.5890791308058669 7:-0.775366087491284 8:0.4983912525892249 9:-0.2976197956132913 10:0.6791258030468514 +-4.394658158733604 1:-0.41628618754613345 2:-0.1472602552309057 3:0.44136102233464025 4:0.011882653940414434 5:-0.6559502840386595 6:-0.4022529016339016 7:0.048402312931387526 8:0.8753776623326166 9:-0.8528247288266961 10:0.6593783978826002 +1.1915739133607073 1:-0.7840827624854878 2:-0.4860418508208426 3:-0.7418773161179972 4:0.129874781837924 5:-0.22631682294184796 6:0.47794208013755024 7:0.5532183426143056 8:0.11879859459306741 9:0.09927630694484524 10:-0.19268618891399636 +2.156192215438919 1:0.44325986644475646 2:-0.6057278708888592 3:0.3943381582091723 4:0.6560336238050575 5:-0.9651308100517204 6:-0.2358219003943678 7:-0.04143043460232465 8:0.8623951169233035 9:-0.4933545255502605 10:0.8990427200454263 +-1.1009750789589774 1:-0.4515707618788496 2:-0.745936099912899 3:0.41307003181926794 4:0.6127760492402428 5:0.9250878169732681 6:-0.6778628527469126 7:0.42794190420905753 8:0.4943969797578971 9:0.7762709104958854 10:-0.6932349268610041 +10.04434496594037 1:-0.0995467494040092 2:-0.7766769414838959 3:-0.6608009972582911 4:0.7075788021090594 5:0.5208396359138381 6:-0.09724033794207299 7:-0.743087245352148 8:0.765372791789753 9:0.3788699859744704 10:-0.6898257995055466 +8.038039859115667 1:-0.5321510657026671 2:0.5571925538006008 3:0.747268102801854 4:0.09079641165917596 5:0.25861122989509266 6:-0.9948187479498878 7:-0.9665136866462685 8:-0.3904629432867681 9:-0.9975425877998279 10:0.32024289816988416 +5.14371929922303 1:-0.4829199170694627 2:-0.5713285263827719 3:-0.43889652467111184 4:0.18478247261988967 5:-0.27374063120041225 6:-0.8069125377696931 7:-0.15497746743367058 8:0.32448521325998714 9:-0.39397735035206227 10:0.08184957956614292 +-1.6848276484379352 1:-0.39250489761445895 2:0.02730338852529557 3:0.9916055514435305 4:-0.07571433435055064 5:0.19024527726403728 6:0.6385182319185971 7:0.32480605537471297 8:0.5807543325220577 9:-0.35642510103381153 10:-0.9060482769392468 +-11.640549677888826 1:0.03707410390488852 2:0.2527049166981137 3:0.4114872952854447 4:-0.8508977901757795 5:-0.42791544663481895 6:-0.9864047295390463 7:0.6023685964407528 8:0.12018443688097036 9:-0.36816249877130414 10:-0.9583147535652901 +11.672104494601319 1:-0.2416258355340175 2:0.6737553249072334 3:0.9041602191361382 4:-0.2123232797997281 5:-0.008255188002961988 6:-0.5151894064136904 7:-0.7341877977528246 8:0.624625272218277 9:-0.6261434804192929 10:-0.5710586715741532 +-2.2960192492344627 1:-0.7457768645184579 2:-0.5954998103421847 3:0.5428846769211537 4:-0.6176587961491775 5:0.46222150678166574 6:0.7852238239427731 7:-0.3614580530629148 8:-0.325840253127059 9:-0.5660596710348922 10:-0.8060263366626401 +5.428302298615722 1:0.8774286357993033 2:-0.23110126319781088 3:0.6264134914476072 4:-0.143015582616014 5:0.350109539755298 6:-0.147747167834422 7:0.05020570422182824 8:-0.5469605849960337 9:0.951112567977048 10:-0.34800121380288185 +-17.32672073267595 1:0.31374599099683476 2:-0.36270498808879115 3:0.7456203273799138 4:0.046239858938568856 5:-0.030136501929084014 6:-0.06596637210739509 7:-0.46829487815816484 8:-0.2054839116368734 9:-0.7006480295111763 10:-0.6886047709544985 +7.314490512652487 1:0.8745354279105222 2:-0.9270067504840309 3:0.965218170323435 4:0.12808957052353698 5:-0.5309399625085234 6:-0.5968520990090951 7:-0.667403236513185 8:0.08154410986660832 9:0.33025488397543934 10:0.03406708067839537 +4.687373993408297 1:0.6731426721418288 2:-0.7111023070261273 3:-0.9849054116048603 4:-0.12831346258317322 5:-0.04095946352836921 6:0.6967001556166801 7:0.8479895229743999 8:-0.35600791972899404 9:0.5005979045264868 10:0.6421341979636503 +-6.82923852156868 1:-0.04849233571020073 2:-0.8505855619911602 3:0.2927180954190314 4:0.5780268040086791 5:-0.22322207765417268 6:-0.8436513934568071 7:-0.3906240514635124 8:0.7258714963093444 9:-0.21695043530813085 10:0.8049335285918169 +-8.24622879369294 1:0.12154833675098842 2:-0.26446415445316673 3:-0.06653791221669247 4:-0.7920694887292259 5:0.6128791496627621 6:-0.6927179137980173 7:-0.24584418172709932 8:0.3557416365779935 9:0.22868636757755234 10:-0.8288196322549064 +-5.090863544403131 1:-0.1535668648046895 2:-0.59868738365189 3:-0.8822518703008675 4:-0.1790505106198006 5:0.9467581256591948 6:-0.0661313762905984 7:0.31263046332923694 8:-0.03628894224569357 9:0.8969599435828515 10:-0.05386674051170348 +-15.780685032623301 1:-0.2568492063716883 2:0.7740976197426315 3:-0.7829158104387535 4:0.8578846037465748 5:-0.6111039318672586 6:-0.26939268282639306 7:0.3659136640533909 8:-0.8205938562638555 9:-0.24945505706767923 10:-0.935948184861368 +-3.9916779937384743 1:0.22925954469403154 2:0.18159238246979537 3:0.05607027262862396 4:-0.3376037702047998 5:-0.10630000583678934 6:-0.7148277241201622 7:-0.08327294541727137 8:0.6532439360618307 9:0.34352364313237294 10:-0.21028242388807916 +8.798748248458631 1:0.509058184822212 2:-0.17532831457577935 3:-0.6387880909085213 4:-0.966194650702529 5:0.7829797328120436 6:0.5470735549914605 7:-0.38312745239682333 8:-0.8890923931840893 9:0.6823342859396513 10:0.9231260597729121 +14.341273640964873 1:0.6996156678090684 2:0.8612833977834464 3:0.9131301694042417 4:0.5199385192744859 5:-0.32605907950755086 6:-0.9816465962348846 7:-0.5939885763232406 8:-0.7730924566676425 9:0.6367821449954114 10:0.10873812383881054 +9.75855501262469 1:0.2933324921347933 2:-0.4652534314332506 3:-0.2940640558090537 4:0.9883453215038367 5:-0.042460731786114314 6:-0.15438550895912062 7:-0.11182397625560592 8:0.7425954283250873 9:0.5063859049644963 10:0.3012211854180429 +7.695200921242407 1:0.3554353390157281 2:0.08707592690448718 3:-0.10815435665633877 4:0.05524046679762784 5:0.8000157491787581 6:0.3756193347272323 7:-0.18659830666742527 8:-0.08168623764933125 9:-0.2551379303720174 10:0.8560030587463281 +26.903524792043335 1:-0.4672678144441864 2:0.868381965588082 3:-0.04748335609643428 4:-0.0908285508827269 5:-0.22436865911994275 6:-0.953965287326564 7:0.40644848732968164 8:-0.33391575325981115 9:0.008337907338700212 10:-0.45597904754961416 +9.87318781117539 1:0.7310287890171705 2:-0.38300115058116324 3:0.5492682498036086 4:0.552016070316655 5:0.3715022458396897 6:-0.3888040017277252 7:0.21348231125683648 8:0.23219558685722874 9:-0.6271161253492459 10:-0.009137052604519136 +7.6930514050666625 1:0.48603550488592284 2:-0.9218820771919889 3:0.17678612698428053 4:0.5110501870908806 5:0.5817010201164554 6:0.4488707800038747 7:0.4977618637956498 8:0.1683214570038094 9:0.17237242672259323 10:-0.5276084644007359 +3.155413914311745 1:0.04582517188512947 2:-0.9060800653779759 3:0.049786270132956556 4:-0.4236784487542993 5:0.6293910028372613 6:-0.7370237043436467 7:-0.4599678991281728 8:0.5317111095323057 9:0.0029525239228334055 10:0.9294876800738165 +-10.18815737519111 1:-0.9023553189306839 2:0.08434165073970856 3:0.7163931103395633 4:0.41749986495957914 5:-0.8190972970472759 6:-0.9996126872234177 7:0.1779075727741255 8:0.18212754689351862 9:0.24628508239298963 10:0.667589863190412 +18.585731475373457 1:-0.8399129036462931 2:-0.10024819268489127 3:-0.5011350892733817 4:-0.7299256348863585 5:-0.9412022985072928 6:-0.245064895931544 7:-0.1032512650854267 8:0.9943863256441088 9:-0.6429371028855466 10:0.062299742931960056 +8.998359297106072 1:-0.16850226855111905 2:0.7262839202089402 3:-0.04876255055071854 4:0.8948164957242868 5:-0.10720585418953132 6:0.2622719447841948 7:0.26433837506661373 8:-0.5143449147399106 9:0.17444585508955002 10:-0.813182163328944 +13.032424230011074 1:0.4014766166181287 2:-0.1710502754125871 3:-0.309850483152607 4:0.255642456909988 5:0.9949117714165621 6:0.12553772251510864 7:0.6412602805648968 8:-0.6225679446416825 9:-0.15867011477056936 10:-0.4970695349036196 +-6.931030745953174 1:0.5151452174260762 2:0.10077064818539072 3:0.9515221270405545 4:-0.21551878535257907 5:0.29152528087481366 6:-0.10995497026133605 7:-0.7872786530801681 8:0.9909149980139627 9:-0.6044617953251021 10:0.4135285912642448 +15.538062451207367 1:-0.493569696351595 2:0.7280914440594639 3:-0.5399160539735497 4:0.5688018985826291 5:0.8296550361854862 6:-0.3519274619833537 7:-0.5536583684230114 8:-0.9648774930921231 9:-0.2649670832738824 10:-0.2337289004188019 +9.499729032920945 1:0.22017490770298553 2:0.7693082799289328 3:-0.7645745307823122 4:-0.4243400515554365 5:-0.7065281515163817 6:-0.9488470141298047 7:-0.7888781431404843 8:-0.38027758953310964 9:0.11329243985448345 10:-0.5636550498916204 +-0.6039115764951412 1:0.3128791250125589 2:0.4690308315665288 3:-0.9819748103687955 4:0.28931283693913223 5:-0.6283983933456656 6:-0.10795935596621975 7:0.7785831799196448 8:0.4453768248295542 9:0.4055410615499917 10:-0.581108383985806 +9.682301463907875 1:0.5039970331368235 2:-0.008965105921562966 3:-0.5415225380115054 4:0.4677111860370293 5:-0.3854089758945243 6:-0.8468317339287676 7:-0.29258253017713587 8:0.7361173598968789 9:0.5722561668394952 10:0.8524030171340933 +-2.8752191903928064 1:-0.45407356732165205 2:0.6563221064539377 3:-0.8938366926767671 4:0.6028173420234533 5:0.6792881349943096 6:-0.6295604812779405 7:-0.21641416912497213 8:-0.8703620515028858 9:-0.3397362922228042 10:-0.0966947467107604 +-28.046018037776633 1:0.9493308195854675 2:0.3285214661535252 3:0.749300278016316 4:-0.006681618268088219 5:0.2936055273341429 6:0.0044706790416966236 7:0.5006172205470896 8:0.38751814960349473 9:0.6069735922707928 10:-0.794612882855285 +2.8752582614589373 1:-0.9443232811926943 2:0.3153126492983107 3:0.6423843271417344 4:-0.09528333043829118 5:-0.2318773828230698 6:0.32597909562645766 7:0.42808555740416065 8:0.2895959316734451 9:-0.5106491076955746 10:-0.2917418155655722 +-2.203945173593806 1:-0.13844025039418084 2:-0.024638102806725293 3:0.6114514176076162 4:-0.6939316676972749 5:-0.036549673716341324 6:0.0942395290460385 7:0.7943411369475493 8:0.7025693796408046 9:-0.21822635487138853 10:-0.6045250179827362 +-5.070655299509993 1:-0.8035156105848074 2:-0.5344928236067734 3:0.6105404604447127 4:-0.7538635525543969 5:0.9836765037886612 6:-0.5700253195942724 7:0.9232380985458313 8:-0.26374377078100464 9:0.9079431466301682 10:0.8404281771949533 +-2.540181413836895 1:0.220453181647285 2:-0.03105792440486077 3:-0.17131282366411926 4:-0.41800060634660485 5:-0.1477564564540963 6:0.055537469469941536 7:-0.8092076926316594 8:-0.29815112444525727 9:-0.20030580647762464 10:0.337865838755971 +19.341342586351033 1:-0.32052868280788616 2:0.954507993011956 3:0.38642226954792824 4:0.9240442034609888 5:-0.46077559741256824 6:-0.7991393493824104 7:0.9396232321156679 8:-0.2486930151964184 9:-0.6256485833035617 10:0.14861843824730103 +0.31398559122529757 1:-0.4684215762946897 2:0.07873308388585198 3:-0.3589594894052015 4:0.14284662079329458 5:-0.8936272055527841 6:0.5647217242826741 7:0.49613233215723507 8:-0.501698787526992 9:-0.46710107378968724 10:0.898517179577361 +12.243117462926584 1:-0.8147610562690222 2:0.21104006948075482 3:0.42405323019132957 4:-0.667965573810795 5:-0.267026607469405 6:0.7949752815579358 7:-0.07664414977654532 8:-0.6023087644686556 9:-0.659375887511856 10:0.459735946423397 +-4.623091296763939 1:0.08405646515942733 2:-0.40009448092691446 3:-0.39831245310544094 4:0.8794137836499942 5:-0.04788565812369017 6:-0.06763019434549333 7:0.41324877265674065 8:0.39746868847324146 9:-0.986729367280818 10:0.7001677710291752 +-5.782162271139417 1:0.29127970805530157 2:0.6712715787317827 3:0.27575757044478477 4:0.31525054647682804 5:0.6905016168465983 6:-0.5195319089267731 7:-0.06598129860341295 8:-0.5142554034519407 9:-0.11546331150946942 10:-0.2934524891698944 +-9.892155927826222 1:-0.7048583334456604 2:-0.46598491327111247 3:-0.02034722477413209 4:-0.663294196316965 5:0.4485329128582778 6:0.4553619594861118 7:0.7154814909138205 8:0.7532937661147989 9:0.020693077287389894 10:-0.23131986644633207 +0.5422764698408844 1:-0.1513298744027669 2:-0.4591544337339648 3:-0.7192219559850723 4:0.21236658135317632 5:0.12050445497328166 6:-0.42411528242712127 7:-0.15103925528861595 8:0.9032115729799512 9:-0.9228817525021624 10:0.2604090001033641 +4.187800872274017 1:0.3084355607627949 2:0.7029638272178733 3:0.34098344122299573 4:-0.5818421369891376 5:0.1332356708082485 6:0.22671316744441716 7:-0.6750469752494854 8:-0.4065302428716193 9:-0.48213803977370073 10:0.17918596677210186 +4.487701812297124 1:0.8352061350259052 2:0.2757393215770836 3:0.9310504392364667 4:0.519503546762708 5:0.5270245209143005 6:-0.08181154800488488 7:0.5148324302455536 8:-0.6680946101511949 9:0.7574060703813035 10:-0.4721334895419935 +-5.150140984417682 1:0.8113709439821006 2:0.21939305063309278 3:0.02109986546311826 4:0.07450107676582762 5:0.723883853128624 6:0.5392035186380486 7:-0.1382740221237464 8:0.9990201540159807 9:0.10429329766137108 10:-0.1365266408862309 +-6.544633229269576 1:-0.08278037549320039 2:0.6982730989138761 3:0.9090685953368327 4:0.6754092061339365 5:0.5889199822482736 6:0.020678619551471433 7:0.47605785660672084 8:-0.49926771127869873 9:-0.28380077002944093 10:0.5282319276258469 +7.216836352055753 1:-0.8510680074642156 2:0.42611818262128476 3:0.593607821624947 4:0.5635067468583634 5:0.2121930523769171 6:0.2708063180622071 7:-0.31491113345871735 8:0.005990053407278095 9:0.8985259402559085 10:-0.44549339042232794 +20.874246167942125 1:-0.53010692413621 2:-0.9897084749945524 3:-0.9083978261828305 4:-0.15581655583739495 5:0.9974035542095165 6:0.9894717992956665 7:-0.7287287537245402 8:0.06425127137526943 9:-0.06684164745938337 10:-0.3600621883071937 +-6.556192430758147 1:-0.7655958349167471 2:-0.08083170734199419 3:-0.8540636958251198 4:-0.09994429443696973 5:0.1734809016500265 6:-0.29563180244063325 7:0.2158497607364409 8:-0.6071644305523003 9:0.8063426715403785 10:0.47092299197899345 +7.252748885335252 1:-0.36403312429467216 2:0.1237451136826817 3:-0.5756427605741237 4:0.7612833636750866 5:0.9350628314096134 6:-0.012087843264624754 7:-0.03742573515965031 8:-0.05210460803183037 9:-0.5333214800203341 10:-0.013320030179712505 +-9.2679651250406 1:-0.5057250557539077 2:-0.41655319851679495 3:0.1897431234740683 4:-0.038318717640150046 5:0.9136495575471062 6:-0.8890525036858237 7:0.40859501498633377 8:-0.8746985847539293 9:-0.005836984002720369 10:0.7838036026237987 +-15.732088272239245 1:-0.8546867577633044 2:-0.3003980324850013 3:0.49649883896876834 4:0.710496747220617 5:0.5848510480601048 6:0.5714826756665468 7:0.5487975165953451 8:0.5654333402837335 9:0.863539315599626 10:-0.9699410102494574 +-0.20412431312519014 1:0.13323548063028934 2:-0.3030177580658542 3:-0.6358920925969869 4:0.3729380701923921 5:-0.8131818118430312 6:0.11567152703716288 7:-0.3645508535812394 8:-0.5487213252460876 9:0.5605886387366952 10:-0.8400308993051686 +10.445759684895373 1:-0.92707280355555 2:-0.9307772570299944 3:-0.11971873660640964 4:0.5140245291069254 5:0.5751145648836897 6:-0.43850910073502347 7:-0.7872208869913662 8:-0.3087975452145404 9:-0.4645849758749403 10:-0.1563641826381328 +3.349708377102383 1:-0.6334394121009499 2:-0.9008086683014112 3:-0.2678892493467009 4:0.7626514243443427 5:0.6406493676995701 6:0.3669245573649391 7:-0.052050629941784665 8:0.6713394117904852 9:-0.11458974566378233 10:-0.25949626043219576 +-23.487440120936512 1:-0.5195354431261132 2:0.8080357948412571 3:0.8498613208566037 4:0.044766977500795946 5:-0.9031972948753286 6:0.284006053218262 7:0.9640004956647206 8:-0.04090127960289358 9:0.44190479952918427 10:-0.7359820144913463 +-11.827072996392571 1:0.9409739656166973 2:0.17053032210347996 3:-0.5735271206214345 4:0.2713064952443933 5:-0.11725988807909005 6:0.34413389399753047 7:-0.2987734110474076 8:-0.5436538528015331 9:-0.06578668798680076 10:0.7901644743575837 +-3.650649176738987 1:0.9665344025238449 2:0.1395514751689353 3:0.954697162791015 4:0.2093601878355722 5:-0.42841737775246336 6:-0.02877209657213764 7:-0.8382526163632971 8:-0.03773878779258388 9:-0.3751775119106411 10:0.6477987464528951 +0.21915863046310957 1:0.25143109618049353 2:-0.06463696557011112 3:-0.3324862332340037 4:-0.7184623449423757 5:-0.8897217937178385 6:-0.7336278194091297 7:0.8547631637534296 8:-0.7582613025929346 9:0.9080481791309838 10:0.9427850135311773 +4.813247597584681 1:-0.4564689661727537 2:-0.4315414033069003 3:0.09676404446694242 4:0.6024645727173434 5:0.20466090997530606 6:-0.09432916868838737 7:0.6402934161890248 8:0.741842551426011 9:-0.343937669190693 10:0.308871619426873 +-3.0700825038127206 1:0.660084046469162 2:-0.02317305725931229 3:0.7567569356692221 4:0.2528834502236612 5:-0.3935091635208898 6:-0.9965507922509653 7:0.9065754202428946 8:0.6988037588300844 9:0.05145737657924321 10:0.4697377584426863 +9.762542323725354 1:-0.036129448543738896 2:-0.8252508992030534 3:-0.752854859129851 4:-0.9027424488033049 5:-0.4763092428375775 6:0.4832492121777574 7:-0.2935697977919014 8:-0.9197908986231211 9:0.8914359296658816 10:0.8688484670974876 +6.690913813146277 1:-0.7649833946109403 2:0.0419327356721928 3:0.5420954694310764 4:-0.7373259510045522 5:-0.9187577877864708 6:0.6431180783847401 7:-0.6272529754533058 8:-0.43356486537110106 9:0.16848266440424364 10:0.3129700315745716 +21.325049167466855 1:-0.36392795201361383 2:0.846518905511275 3:-0.26361421923150097 4:0.5140384860444887 5:-0.9147771624497878 6:-0.22044646197773576 7:0.14099760779666948 8:-0.546631395802236 9:-0.4345465263406878 10:-0.2759894364167672 +0.41237529640734055 1:0.05016964684797287 2:0.21708512805176072 3:-0.9444942733586354 4:-0.6118772896807114 5:-0.18053631846913665 6:-0.06752556529755416 7:-0.0081819952134361 8:-0.7774039956687315 9:-0.5548994336153177 10:0.7510833121912588 +-15.056482974542433 1:0.6012054064354875 2:-0.6127014811673221 3:-0.8356741843949218 4:0.19830469052767397 5:-0.07726493085289698 6:-0.5756891943805014 7:-0.49010583357941884 8:0.7493759119974515 9:-0.7828994218436376 10:0.6154265137741459 +-2.109441044710089 1:-0.5757976103755722 2:0.3686657403505862 3:0.5418762444017706 4:-0.5896052565388463 5:-0.1000712585735879 6:-0.8114188394866342 7:-0.5863884932327266 8:0.28289838755838015 9:0.5378646921099333 10:0.5063780890366179 +-5.249715067336168 1:0.6828022788286754 2:0.9044668986252975 3:-0.6010464361571437 4:0.8416122052398811 5:-0.9846446498408039 6:-0.3777762313579811 7:0.5763775880953983 8:-0.07608009385213488 9:-0.29576023599575474 10:0.8845728751981716 +6.907770824878343 1:-0.9751352215365647 2:-0.8297271715190588 3:-0.07240311280415779 4:0.4796310183582191 5:0.358213469979769 6:0.4628020211207058 7:-0.9753405605972942 8:-0.765583403709019 9:0.5623611232648877 10:-0.14725965272406616 +-9.299021854126096 1:0.8784076266914045 2:-0.8314918563417382 3:0.8701529449600536 4:-0.8070129727442199 5:0.07396877198841345 6:0.0040889707225901795 7:0.40529205456687145 8:0.6412485325027342 9:0.1443450351498905 10:0.404997568726581 +10.95643670126225 1:-0.37321642594676097 2:0.47766490569544473 3:0.9201313123144423 4:-0.649393433578801 5:-0.9084894063674787 6:-0.2547160991750408 7:0.7674649994523459 8:0.646056370118979 9:0.6014100713287893 10:-0.15130291862509182 +-2.6397202393123336 1:0.3285252466844373 2:-0.2714281159811125 3:-0.5869561846815805 4:-0.5643935541712441 5:-0.7285201267315389 6:0.6502951074428092 7:0.8611880383193904 8:0.6380425291162128 9:0.5118538704085516 10:0.4012684110865874 +12.521131042032012 1:0.4843931319727355 2:0.06440853455169626 3:-0.6151259240105509 4:-0.4180928328467284 5:-0.4607061773323424 6:0.8104775289268906 7:0.3284199695768064 8:0.8425028998495565 9:-0.34822319854822825 10:0.1969239149176112 +-16.151349351277112 1:0.7787909191620395 2:-0.5219981442072688 3:-0.7230569918898555 4:-0.05707801168212101 5:-0.8134225824740247 6:0.09950684183685454 7:0.6261274830059296 8:-0.9502006765164366 9:-0.6724983095526844 10:-0.600347212281825 +-5.039628433467326 1:0.7680701397575322 2:0.7956844224408437 3:0.4131717201035916 4:-0.3127895385265915 5:0.7226571953995224 6:-0.06845863083031967 7:-0.1007291660029832 8:-0.9130249132342207 9:-0.4605180615474036 10:0.42093879298156 +9.007596502870785 1:-0.6562175566238462 2:0.08420074013217049 3:0.589801949672486 4:-0.11964901133703987 5:-0.8145711913860048 6:0.43854302140351065 7:0.5992967124729605 8:0.253745043289755 9:-0.10742030998120033 10:-0.5993228348160153 +-12.41094640284016 1:0.31035917086763765 2:-0.8053417167237813 3:0.5754655536186164 4:-0.3645388095106201 5:-0.9135176753316416 6:-0.8690739610562535 7:-0.14039224825138197 8:-0.7112835675593987 9:0.25762942117230825 10:-0.9483300117501923 +-12.130353212287929 1:-0.41404309625298485 2:-0.7695984204591535 3:-0.44569447239245275 4:-0.3168863099965644 5:-0.26669244730409036 6:-0.33484042698895755 7:-0.41062396946367685 8:-0.09075804785640385 9:0.8511367190902208 10:0.021918606255194595 +-15.375857723312297 1:-0.9794952880997945 2:-0.9547237660069134 3:0.2460912345929791 4:0.3174335823329406 5:-0.23758562926743054 6:-0.113610303129287 7:0.18292675847568063 8:-0.9656446754474337 9:-0.58300134324846 10:-0.6689602908128025 +-6.397510534969392 1:0.440780662587545 2:-0.03737991637410243 3:0.9506435891605849 4:0.8177486462589998 5:-0.2917628929963241 6:0.42365289098031034 7:-0.4280555544979745 8:-0.18388426864865903 9:0.5057230088452542 10:-0.1699163749308643 +-9.789294452221961 1:-0.25066699970459694 2:0.1772977344415987 3:0.5913498268900952 4:0.6293756431864681 5:-0.6430441015863757 6:-0.7238519180293621 7:0.13639541626580498 8:-0.6620281401715837 9:-0.9515237061912034 10:-0.4333426289849791 +-13.15333560636553 1:0.3491978525665129 2:-0.4067353159374012 3:-0.8677040612253524 4:-0.5757086910974862 5:-0.3186886816681207 6:-0.06609938943414573 7:-0.5419747642754873 8:0.9632759660044383 9:0.2673520823110991 10:0.36463236596724546 +2.2307697392937795 1:0.12285527276472785 2:0.8938323722714365 3:-0.16995870341610209 4:-0.3298643049714254 5:0.16781582791954253 6:0.42381594687105895 7:0.9245288214717629 8:-0.08709025093361689 9:-0.14227085487682722 10:-0.2888302862659746 +5.892885365508635 1:0.10116053019915738 2:-0.41641547074900154 3:-0.3750004290914961 4:-0.5619470211369917 5:0.33343039544460384 6:0.46766042657994733 7:-0.6937940929321615 8:0.7044604392055189 9:0.8879353764416567 10:-0.5490902425042639 +-16.692207021311106 1:0.9117919458569854 2:0.628599902089868 3:-0.29426892743208954 4:-0.7936280881977256 5:0.8429787263741186 6:0.7932494418330283 7:0.31956207523432667 8:0.9890773145202636 9:-0.7936494627564858 10:0.9917688731048739 +10.454641756541454 1:0.3490213088098768 2:0.6103387992494194 3:0.6753935651135747 4:-0.39560763769937934 5:-0.3042308221531884 6:-0.9283481899557042 7:-0.7904038212853011 8:0.37488335848537346 9:-0.296477977723397 10:0.30894819444660304 +0.08978797103855778 1:-0.13445409764877803 2:-0.6404150831493631 3:-0.24740260669490133 4:0.031151119464385646 5:0.9207882173498612 6:-0.6146471129497393 7:-0.9736175690408087 8:-0.2673180325645341 9:0.5800384183301572 10:0.479811220263183 +1.7362099941626894 1:0.5171681395917551 2:0.6177735922313075 3:0.6446678302226738 4:-0.5731769722311459 5:-0.2686270617709168 6:-0.6048534221658814 7:0.7002124303669326 8:-0.1479765297345712 9:0.009254061109394307 10:-0.31519081920853287 +-1.0349488340235453 1:0.612980711993536 2:0.05771318707554962 3:-0.10821368362160744 4:-0.8755559420458141 5:0.42566546089913326 6:-0.7966341558699277 7:-0.45253617234374466 8:-0.8289517557653971 9:-0.8968075137250837 10:-0.6325457096866376 +0.10157453780074743 1:0.9143592240573388 2:0.06419631741815457 3:-0.9961326744227916 4:-0.47174548800139715 5:-0.0821464027819967 6:-0.5495006555498168 7:-0.5627911401420294 8:-0.43426056724099005 9:0.892026786364895 10:-0.23546485121284055 +-12.92222310337042 1:0.218687524173371 2:0.013626751799176162 3:-0.8372219908323961 4:0.6197296846266354 5:0.7429130827811232 6:0.48009972886541896 7:-0.35667717521227904 8:0.18337067878780533 9:-0.22935396092245197 10:0.4076715024284059 +22.923352376063196 1:-0.7522075505725567 2:-0.20686029838909326 3:-0.1386664769095396 4:0.157117595808127 5:0.9462377653889174 6:0.9182504509330662 7:0.18170057747293833 8:0.27735387813088863 9:-0.6355799944714868 10:0.9764849106195284 +-6.132450015997121 1:0.2822534275343054 2:0.2625905791399692 3:-0.02565260641304068 4:0.4891221076432757 5:-0.6426178913585772 6:-0.8999539149461033 7:0.12659507663825287 8:0.5889572439755832 9:0.49107548332672857 10:0.47595749470452 +-9.173693798406978 1:0.4430245286298278 2:0.9923116639471541 3:-0.5609082824097824 4:-0.36542266258313916 5:-0.5814039716882617 6:0.20413852042674874 7:0.6097541611931963 8:0.5743002479324253 9:0.4735459963431561 10:-0.053969823043886755 +-5.814408490931223 1:-0.9206287328000513 2:-0.48298486023273157 3:-0.8477202041890262 4:0.5801385102362351 5:0.7146074564553095 6:-0.5987672678579339 7:0.6829077928212723 8:-0.3522788540815065 9:0.7729595638821951 10:0.5264904880591215 +6.474329501040298 1:0.6914309300550991 2:-0.4507700505202725 3:0.713821440501512 4:0.41599059910235847 5:0.507160951750409 6:0.8635615811574222 7:-0.6235518270244333 8:-0.5336201820384283 9:-0.7989630679361768 10:0.837293162455248 +6.984517471584806 1:0.16745919469723392 2:0.018033079961716103 3:-0.7339201095541323 4:0.17042828693740697 5:0.4493471632580528 6:-0.8938445962323078 7:-0.3206968104792325 8:-0.616617071238893 9:0.9327878222034172 10:-0.6575294247048245 +-12.479280211451497 1:0.9769767754725367 2:0.7706430863248943 3:-0.4506244622476816 4:0.12921761745628713 5:-0.0697804449658812 6:-0.7702703569987461 7:0.017734558413919688 8:0.7216294158911261 9:0.42547357862241886 10:-0.9001915116155741 +2.8363866587728186 1:0.11478724114928918 2:-0.4679790550082039 3:0.2344912687736711 4:0.5524878060045462 5:0.5252859884051309 6:0.5080674087215156 7:0.5010449021825665 8:0.048046765816400105 9:0.06654581719548891 10:-0.5801934713347348 +4.186809777233374 1:-0.02335342201396018 2:0.9035437912091193 3:-0.9283585631882163 4:0.454351316397237 5:-0.6948564428085262 6:0.11495485234890368 7:-0.23683956078769963 8:0.6442534752881419 9:-0.013866407845647188 10:0.23369602940650736 +2.8235031660626415 1:0.5609344938188046 2:0.3449103464885612 3:0.03972169049525687 4:0.31858762565827137 5:0.4409953589124853 6:0.22836189275697016 7:-0.1497811991899889 8:-0.23248048920679265 9:-0.30066618281100177 10:-0.9247232456911632 +6.96223432848425 1:-0.8160398553437558 2:-0.8212180893749699 3:0.7728655115832999 4:0.02387973088796369 5:-0.043499804905828166 6:-0.6997726250046865 7:-0.8686633773265577 8:-0.12597318402253976 9:0.967018116368416 10:0.5951339624149812 +4.669684795838683 1:-0.32226903644852833 2:0.5465858078942492 3:0.5228467793266189 4:-0.013157722224545143 5:0.5810668818928995 6:-0.1372653090293532 7:0.6446157527288279 8:-0.06005754873230629 9:0.014302180040152379 10:0.43474245441042636 +16.112744845653285 1:0.37257742858083365 2:0.19398954512844124 3:-0.11860882189887478 4:0.6492510749703395 5:-0.41273736981203313 6:0.18643017041815835 7:0.29136917186214384 8:0.47602883023389 9:0.7126916980867937 10:0.48462508659691483 +-9.196003366226202 1:-0.7263358951920722 2:-0.8503799288093836 3:-0.3120563620589105 4:0.3925562655164563 5:0.027666662972283484 6:-0.35173134138805406 7:-0.32703527910354757 8:0.3060102722285065 9:0.8609161725740202 10:0.33394557004432923 +1.242972458167591 1:-0.9029238804456814 2:-0.6392681059531908 3:0.8940879647942577 4:-0.8807357173896475 5:-0.13628130467470512 6:-0.5487534785116224 7:-0.40270307148061346 8:0.09152108686997096 9:-0.20745066734844642 10:-0.20624830574384978 +3.453659210660726 1:0.2710596844435682 2:0.6510497900145247 3:-0.2899158136103117 4:-0.13531811694554707 5:0.6965847786422426 6:0.9105343028780231 7:-0.007340232468413754 8:0.7672537187738411 9:0.3538906829188173 10:0.35387524540947646 +-0.48115211266405217 1:-0.17943755364759517 2:-0.1384979591151625 3:0.8425773648797268 4:-0.43234064993405097 5:0.919754442523921 6:0.8390197802990036 7:0.43890653121452683 8:-0.7647648217789051 9:0.14770258954363835 10:-0.6681813635676657 +6.965069440749298 1:-0.9158261471030473 2:0.5228494114644282 3:-0.07760531122743153 4:0.6154296244963067 5:0.5231830145381096 6:0.4892535590799165 7:0.1987053183082137 8:0.9995670294711712 9:-0.2020375688074112 10:-0.7853579334836087 +-1.6896486293598596 1:0.4638529147853421 2:0.0953805943546191 3:0.8506904243225251 4:-0.028262644692445438 5:-0.9462342015500664 6:-0.6934738957112123 7:0.601125018257533 8:-0.04871041957758315 9:-0.015245062056267411 10:0.6119856200040805 +-1.763729644326212 1:0.5376618752928528 2:0.8062119856717131 3:0.44996834959923593 4:0.9917728248530817 5:0.5974717482179492 6:-0.406972851600659 7:-0.8523198502065281 8:-0.3076377139692321 9:0.9099974915864462 10:-0.43374966692373484 +9.012829566937228 1:0.6885456531832366 2:-0.0631164354373237 3:0.8394182300770314 4:0.7207913383891218 5:0.4715324450375691 6:-0.34417503908167757 7:-0.31448279255342126 8:-0.020591617987411936 9:-0.37668573574418107 10:-0.6528048324896532 +-15.951512565794573 1:-0.6112828771933607 2:0.4867007149846869 3:0.863494046941478 4:-0.7292072742454481 5:0.6338749652624007 6:0.5980798993978542 7:-0.5119002889878654 8:0.8394383182101366 9:-0.1412423080445726 10:-0.15838730884968655 +-0.29622788243318465 1:-0.9436253326661384 2:0.2907259958032098 3:-0.1530538226933904 4:-0.6174176535420375 5:0.8209632215649141 6:0.5060548803172731 7:0.8212448453211292 8:0.33506684706740386 9:-0.5408309869188785 10:-0.8105966349150977 +-7.683213587039055 1:0.2525015766703558 2:0.6417869320191234 3:-0.7569571597336913 4:0.5265130776924394 5:-0.03992944660560949 6:0.18292946303778823 7:0.4286344960738724 8:0.9158523573288766 9:0.5039796366711773 10:0.27660486075533797 +3.9061298856792797 1:-0.6501789225392032 2:-0.6040685518173872 3:-0.6448094322678659 4:-0.2019498832769746 5:-0.5302977370883424 6:-0.010754341856880067 7:0.8791702222974846 8:-0.2283571791337704 9:0.4726320486679656 10:0.3413255179758332 +12.928385148211825 1:0.7793178379505685 2:-0.5207562047491976 3:0.37253320760898934 4:0.7540757518052998 5:-0.679378421540417 6:-0.11966022036636881 7:-0.4317798870297489 8:-0.004211291952602059 9:0.39024653887361693 10:0.45391057946097146 +5.787566514603203 1:-0.20596730554338039 2:-0.8840796727164746 3:-0.749416279057892 4:-0.5511023306046077 5:0.9941631901218697 6:-0.09907966722992234 7:0.701617914811792 8:0.9696055014561289 9:-0.7083648075748707 10:0.5781111533720358 +5.701262468657861 1:-0.7066995012593675 2:-0.6756815056791965 3:-0.5720277255842998 4:-0.09218662060241067 5:0.21494136076896653 6:-0.37012884573008153 7:-0.6828277646796448 8:-0.10038134655965236 9:-0.46253754509583356 10:-0.20813933595648115 +0.9473494330088033 1:0.6876806675510589 2:-0.9530860102792402 3:-0.4043172626863887 4:0.6696455505098386 5:0.17863581804857254 6:0.1944646561635497 7:-0.5283662172535679 8:0.4872263841818012 9:-0.2882651789318431 10:-0.06293411605141874 +-2.6834375589185675 1:-0.22376759986120187 2:0.36555755546798885 3:-0.5223502955721961 4:-0.20702347869224624 5:-0.7745351063999764 6:0.22879328233099971 7:-0.5440007473902635 8:-0.6959483071829207 9:-0.131433881760733 10:0.2764225554693165 +-3.2766108642276146 1:0.0304613976530983 2:-0.3148062986719251 3:0.24950420590071953 4:0.7152023826801459 5:0.9656885739650887 6:-0.3210562623763835 7:-0.7305896664502614 8:-0.49074917893875836 9:0.7802670253347352 10:0.8667409958355992 +-1.1838791995691869 1:0.06642047806096318 2:0.5336148776806793 3:-0.6199614859883396 4:-0.15342280723497237 5:0.8407250402808968 6:0.7060811811107444 7:-0.2913182140909305 8:-0.5925203360011633 9:0.22644925021629692 10:0.42395071889002467 +-1.5856680515554806 1:-0.8724712788102853 2:0.11445744032031424 3:0.5483166457680566 4:0.9469521544884028 5:0.2541682828467746 6:-0.436750733871873 7:-0.9001249399695319 8:-0.7555793441458385 9:0.06946992897983018 10:0.9724148045760346 +-13.039928064104615 1:-0.558607026518148 2:-0.7356765018678253 3:-0.7547644426290201 4:-0.24898664843938745 5:-0.3606374046883567 6:0.5836652368902306 7:0.8497678666873467 8:0.21331875915717635 9:0.3558733809635668 10:0.9642603628738968 +-17.428674570939506 1:0.8562209225926345 2:0.7077202100653552 3:0.7449487615498371 4:0.4648122665228682 5:0.20867633509077188 6:0.08516406450475422 7:0.22426604902631664 8:-0.5503074163123833 9:-0.40653248591627533 10:-0.34680731694527833 +13.886853032969585 1:-0.6354915752033683 2:-0.9132338112681755 3:-0.4816479770266455 4:0.5448417181244594 5:-0.6250746297187781 6:0.7410618768880199 7:-0.18029029550083675 8:0.777358236920447 9:0.9625064189449102 10:0.048040935468046 +15.61684729251139 1:0.2980237970192188 2:-0.8160931971814265 3:-0.29649852157138445 4:0.3896688599904572 5:-0.17552110506337826 6:0.8721328328445139 7:0.48984799668438916 8:0.9984496052876473 9:0.9665885195526289 10:0.8966559812150274 +10.33625540376971 1:0.09939495068155724 2:0.9790332181038015 3:0.9483428886275702 4:-0.5717299810793317 5:0.4876405069057712 6:0.163962913892302 7:-0.4095537988924203 8:0.8608269751255508 9:0.010028680058212114 10:0.9095786494455713 +9.706032970113723 1:0.7687898546315146 2:-0.9825109379412285 3:-0.5423211794439926 4:-0.3099509487314134 5:-0.11561305536236333 6:0.9012327035409926 7:0.5257495475790148 8:-0.33804422025989433 9:-0.144428735681567 10:0.28019332199039604 +6.189043888072968 1:0.13246655756059478 2:-0.751192382628302 3:0.2233421456265161 4:-0.1933575076984373 5:0.8681727702736863 6:-0.7656847407654899 7:0.1033145549916572 8:0.33909210370257403 9:-0.22241363302770267 10:-0.14479004187830435 +-8.680225911784335 1:-0.07718769939880432 2:0.6702228057326558 3:0.6647810334933819 4:-0.05115658747070784 5:-0.850780588302118 6:-0.040961453376221924 7:-0.8407690297644956 8:0.33775829053563156 9:-0.45421556034898547 10:0.8238500771967823 +-9.42898793151394 1:0.8925906426831107 2:-0.6771269725125597 3:-0.11635105688280678 4:-0.7266044201050157 5:-0.6902918845825077 6:-0.5911234800910024 7:0.49395074569300657 8:0.43660804414878274 9:0.8736983081269782 10:-0.8001177058312081 +8.486245765579415 1:0.5614295382716652 2:0.3972427851719582 3:-0.276268504977494 4:0.7803448249454739 5:-0.358957923558495 6:0.3477822689529795 7:-0.7944805581842691 8:0.8356932134547437 9:-0.4783293647580624 10:-0.2522633417723845 +-1.8722161156986976 1:0.11831037290857482 2:-0.7309091607574014 3:-0.7339122716951587 4:0.2046641765436359 5:-0.9914679283125301 6:0.13518339528098555 7:-0.9760821540963867 8:-0.6080636193563043 9:0.3890502262427238 10:0.33864957953815145 +0.5122357093733743 1:-0.9555852441641726 2:0.4754771858792488 3:0.3743376249200432 4:-0.2651772997462427 5:-0.7915484529586028 6:-0.7575915279708862 7:-0.10432268807273859 8:0.021604934223709238 9:-0.6458011732912265 10:0.40773716196391674 +-18.845922472898582 1:-0.6031480148285926 2:-0.8736524730197766 3:-0.311456616524979 4:0.420921703897325 5:-0.2904011177124777 6:0.6683252350591937 7:-0.3436202976676894 8:0.5023604359385605 9:-0.33056149241985633 10:0.5168854058825227 +6.492106438811399 1:0.7824832256885428 2:0.6105456307389117 3:-0.0436873997963223 4:0.46730493583332855 5:0.2057529813440686 6:0.5738310686722767 7:0.6307964411259019 8:0.6208424783086652 9:0.8931894299284251 10:0.7164648197763028 +-1.6472226859532182 1:0.8854767145642171 2:-0.8175744681485637 3:-0.14894858038610903 4:0.9667400540136402 5:-0.3575837217508149 6:-0.9211342680517054 7:-0.956785876301889 8:0.6558217028031554 9:0.8014538160668165 10:-0.9475520920917395 +0.185861229793925 1:-0.8181719548530746 2:0.9990094335332504 3:-0.8195848911987829 4:0.6991933015233858 5:0.07295718417836583 6:0.5968996100546737 7:0.4871410306452193 8:0.2980483098540927 9:0.779953293728507 10:-0.7978867112395516 +-5.973450525185694 1:-0.975435413991927 2:-0.7832951303253313 3:0.5098999023442101 4:0.46795978867990007 5:0.2538986807863044 6:-0.8182887550010198 7:0.8335391734637112 8:0.4286082996234335 9:-0.1726765956719154 10:0.7649845978453362 +-12.773226999251197 1:-0.383327656965585 2:-0.9439560491389036 3:0.25039001869622446 4:-0.9342091044843222 5:0.8711023711291135 6:-0.6027135241543655 7:0.9456874780319795 8:-0.243290468946338 9:0.625765915285031 10:0.5160550067618355 +24.290551295953957 1:-0.8368553572749229 2:-0.5859456648150321 3:0.873779532007048 4:0.7462623178738954 5:-0.08133011570245352 6:0.36767541461776676 7:-0.33129619282275047 8:0.6104289727615573 9:0.9416581563055089 10:0.18201841676606856 +14.490247980976621 1:-0.4765937762114507 2:0.16430711839945555 3:-0.526776940706293 4:-0.6802269991653915 5:0.40748236413299344 6:-0.500290139207977 7:-0.31915972151663885 8:-0.4586068416002418 9:-0.15572660263944127 10:-0.32925702602833073 +8.377230871265601 1:0.44141613060964846 2:0.1582267687752743 3:0.8760950367284166 4:0.40434058393690364 5:-0.7063758409891474 6:-0.616055773516162 7:0.996372393127579 8:0.6142084876085476 9:-0.528320587432094 10:-0.2815909691094802 +-3.2987560995836653 1:-0.4600479783378091 2:-0.04201794336103326 3:-0.8934505203905587 4:-0.44991326751905536 5:-0.5220579476363783 6:0.46060949186328703 7:0.9169289030735643 8:-0.022458426893944283 9:0.08100795210565637 10:0.5726732415540354 +0.3422568955736137 1:-0.9888686059817204 2:0.22752298580182706 3:-0.5048696915520232 4:-0.059433420464226616 5:0.7823831512651716 6:0.9865977573980389 7:0.9164100011124972 8:-0.3638554550863984 9:0.3038282907667611 10:0.4652367033461571 +-8.24116881862084 1:0.7565819250331731 2:-0.3733277500524168 3:-0.8841150081071696 4:-0.922282989989148 5:-0.041520813551309876 6:0.8615967014876558 7:0.8474207144091339 8:-0.7518437864641427 9:0.45076605239968837 10:-0.48912984167595375 +-4.367083147104942 1:-0.276459380002813 2:-0.957555271384241 3:-0.3761632810202544 4:-0.3897414804149022 5:-0.3133861519856074 6:0.0777990809172171 7:0.6638552243422928 8:-0.3477312155364247 9:0.5934885465182675 10:-0.5238903641193555 +1.9280240152322783 1:-0.40051093785549696 2:0.5070348672240661 3:0.7506759969575532 4:0.5042104954516786 5:0.9959688260926507 6:0.4657024999761399 7:0.910611131925299 8:0.9836517468598804 9:-0.6263172749113686 10:0.16955852322929155 +8.918138317441574 1:-0.22407391224687023 2:0.5545084933214972 3:0.6335932367683528 4:-0.2786481116648991 5:-0.9549992830441785 6:-0.5577873948545062 7:-0.960657200286197 8:0.3709573488946196 9:-0.9191180485753339 10:0.5033478020271929 +-5.657796797481157 1:0.6359910361030725 2:-0.1742637774815281 3:0.39699327107265137 4:-0.9841991491194473 5:-0.622093571871533 6:-0.5433497301426455 7:-0.6731178481686009 8:0.930615153085582 9:-0.3065877908950827 10:-0.5456093749639228 +8.697079562319692 1:0.4815820396629933 2:0.1173457441514223 3:0.7313645402039386 4:0.3354835387237334 5:-0.10300554535074702 6:0.5116687640761355 7:-0.8850803659104614 8:0.10654026377571157 9:-0.864976708975602 10:0.01345035085413615 +0.033954684723234596 1:0.6703241653088159 2:-0.13447915740201166 3:0.026022550037831937 4:-0.5145659862194116 5:-0.6963587636078901 6:0.652083884947352 7:0.22644722530715278 8:0.2671580129293405 9:0.9659035105360283 10:0.9547989197693989 +7.359108382166921 1:-0.6855762478384229 2:-0.7543318537260015 3:0.4772611975128618 4:-0.5588002332845741 5:-0.24271386844336496 6:-0.28595644325868896 7:0.8732728098501104 8:-0.8026384804471058 9:0.7589508830210041 10:-0.9992933613402135 +4.953597303754355 1:0.8915633023548608 2:0.04688596266450751 3:-0.26866754730613374 4:0.16694236975718102 5:0.23465297255622608 6:0.36488427850844407 7:-0.06717041145276781 8:0.9470029805221898 9:0.32483835237272674 10:-0.7892521260150298 +0.683536559775105 1:-0.32176084249781556 2:0.5446298870866526 3:0.4095848716057642 4:-0.42579711490120187 5:0.4482850543749355 6:-0.0982243826242506 7:-0.9190317048427039 8:0.06234509402976718 9:0.21327512416175054 10:-0.38023673796734525 +-28.571478869743427 1:-0.4597184465402242 2:-0.5489429386926741 3:0.33422914572951634 4:-0.15992695377395516 5:-0.7310003311728188 6:0.18241063863467488 7:-0.48385214010599453 8:0.08139879039334552 9:-0.8401239538877046 10:-0.8896372220209929 +-19.884560774273424 1:0.4619217451285318 2:0.28157115824800005 3:-0.3829811521605375 4:0.5802544015450464 5:0.1117061271473403 6:-0.8926034502584623 7:-0.34862293810401956 8:0.2733254857260612 9:0.6514176550598809 10:-0.02758604919357066 +-17.494200356883344 1:-0.4218585945316018 2:0.15566399304488754 3:-0.164665303422032 4:-0.8579743106885072 5:0.5651453461779163 6:-0.6582935645654426 7:-0.40838717556437576 8:-0.19258926475033356 9:0.9864284520934183 10:0.7156150246487265 +-15.86200932757056 1:-0.6341453831788726 2:-0.9259180639727085 3:0.302702923864538 4:0.749555004323947 5:-0.7932989575334761 6:-0.5620972938631934 7:0.020542041027870717 8:0.11610338700447698 9:-0.7912600154897766 10:0.5108307672038874 +9.027804254487519 1:0.1746878011084212 2:-0.5872807344913673 3:0.6018547246457264 4:0.5106104933121229 5:0.7329523371170135 6:-0.40058771577765895 7:-0.48753463550174025 8:0.34308791976318 9:0.3407668956765344 10:0.5964472848798394 +15.949172086880687 1:-0.7790584545657173 2:-0.017224094786103317 3:-0.0974907790179953 4:-0.10287391996036166 5:0.6007953354774878 6:-0.7032497754397848 7:-0.36068070856329437 8:0.021391994204512432 9:-0.6509100388083549 10:-0.5410899936281377 +-6.151586699415245 1:-0.5318094974022525 2:-0.830796057445983 3:0.603828597318087 4:0.6660892552257192 5:-0.18529748408390523 6:-0.47166833767648986 7:0.592915541856605 8:0.9944601563352204 9:-0.6981606574244703 10:0.34942553665003584 +2.010398523297265 1:-0.9293899922307269 2:-0.07588009904844029 3:-0.8500855420709359 4:0.12191867923536615 5:-0.528778681165414 6:0.3117086447237414 7:-0.4222963938187163 8:-0.03247894950300623 9:-0.05387792412717962 10:0.4053568741659812 +-6.749023248121471 1:-0.9875370165216966 2:0.7137693455001415 3:-0.2510160963160164 4:0.8732150877079123 5:0.49658934612905314 6:-0.9817012857861731 7:-0.2045309437850289 8:0.7562713668333418 9:-0.6787434327188155 10:-0.6147932888026117 +4.452639829999693 1:-0.35256148944834176 2:0.7581152951164591 3:-0.37755890552299265 4:0.9480813371197343 5:-0.3419340388717347 6:0.3487602851799074 7:-0.5576726724270562 8:0.4899696188087421 9:0.563074979676983 10:0.7865891460062227 +-4.938733988900586 1:-0.4108386466193119 2:0.3287655432069885 3:-0.5853553038038923 4:-0.6480591422742821 5:-0.4787998161299789 6:-0.5828003484675421 7:0.42835744317623003 8:0.8378098987706633 9:-0.5645180498703375 10:0.28981512694646705 +-3.373242544176224 1:0.04989033652617936 2:0.6575826440927308 3:-0.24028051935833128 4:-0.6649808138961095 5:-0.6530198970442704 6:-0.19331254127919362 7:-0.6743004878881749 8:-0.7214986105015062 9:-0.30648035516261385 10:-0.6455097687924254 +-3.2843694575334834 1:-0.3548536057581908 2:0.7350125943559394 3:-0.3635282827378974 4:-0.8552820154885781 5:0.9140879208466111 6:0.21870365067770892 7:-0.17738543429561382 8:-0.052851966578491005 9:-0.36066059517759097 10:-0.9020765799355679 +-3.277146077677404 1:0.910961221014513 2:0.4302525202590246 3:0.11079959840001119 4:-0.3614188274820125 5:0.5080231397310961 6:0.013940825892631237 7:0.33583012240022403 8:0.5008797094229163 9:-0.663083147090173 10:-0.0865028013627418 +-0.202246147968096 1:-0.4929308143227653 2:0.8374300027105082 3:0.08763999085193186 4:-0.499738438136623 5:0.5926071511295365 6:-0.5135396038023627 7:0.6946715869746543 8:-0.5184428793490325 9:0.21753085495829239 10:-0.33796308746585235 +-7.1237150573506955 1:-0.8506203499039495 2:-0.6581804183622855 3:0.6484205342724825 4:0.013914696389758285 5:-0.6214530117645831 6:-0.011163110491807293 7:-0.6025372583334574 8:-0.0371573886520411 9:-0.7933455929226487 10:-0.38653838674273455 +6.298226129171093 1:0.7304191211928768 2:0.8128475475660479 3:-0.03161148630216015 4:-0.6018899317958344 5:0.19277055729934367 6:0.3002272616310928 7:0.949169758830406 8:-0.1011823256970481 9:0.16093341376629966 10:0.9596833606094763 +14.906594657519511 1:0.5053240355803015 2:0.6775698974866082 3:-0.6194771000646291 4:-0.02876927004033525 5:-0.5481504206112477 6:-0.9239150546263386 7:0.471216755072994 8:-0.0027794620943384363 9:-0.8954411386878227 10:0.8991742143686698 +2.1710965297686267 1:0.4578509053930304 2:0.9270194505165124 3:0.22470373699901236 4:0.21526179917432753 5:0.5299563895862103 6:-0.5824108997775908 7:0.03801922095671095 8:-0.5164033454609385 9:0.4370246809487237 10:0.6514133050988229 +15.05806598279517 1:0.48645077410559057 2:0.7821442063987365 3:0.1943681666933883 4:0.8289246958621577 5:-0.08034311437806041 6:0.03709694472527203 7:-0.895481297246602 8:-0.42921579749551664 9:0.5447075872378688 10:0.844397849728866 +-0.4683784136986876 1:-0.5083135683360327 2:0.626070365769088 3:-0.8737725909401557 4:0.725622293853621 5:0.0018794384199978253 6:-0.9343604622552886 7:0.6655593328822609 8:0.47501755618845753 9:0.8388618477210947 10:-0.5143806767304449 +5.823027255871114 1:0.08635467091841886 2:0.6314532702073175 3:0.8862069437865836 4:0.6542025864928516 5:-0.6846784290231471 6:0.048487096050569445 7:0.30828004933669395 8:-0.49438881988995687 9:0.5706936923061823 10:0.037705651885639346 +7.03779380408974 1:-0.07193682621291098 2:-0.5816975957307158 3:-0.8426927090342973 4:-0.37504851992255306 5:0.4473129018316815 6:0.3101938194888525 7:0.6160050428837607 8:-0.913998555949695 9:0.40461966540531313 10:-0.7581141330823786 +-9.770500546345563 1:-0.31358873581579894 2:0.11771478839130278 3:-0.3404842110585631 4:-0.0604362797252429 5:0.2159524972176814 6:-0.24737863017398087 7:-0.8541428610709716 8:-0.06753562283135062 9:-0.11567537916769255 10:-0.5606246203677223 +20.000154367451547 1:-0.344717847914646 2:0.8454969480099985 3:-0.58856299370874 4:0.5884510299634649 5:0.49162879631128553 6:0.7958075013181658 7:0.7781911267315837 8:-0.6780885011989877 9:0.9797694629597928 10:-0.1872163682079866 +-6.239848349456753 1:0.9132793720646253 2:0.1680340663118458 3:0.01740115925682284 4:-0.26580395408599133 5:0.28551914590761074 6:-0.9939706142381568 7:-0.8740927279520219 8:-0.8731218126652498 9:-0.10993630739903892 10:-0.3069565039708746 +-4.173072569004537 1:0.7864835254860851 2:-0.5614522227484218 3:-0.7718396381376464 4:0.49508673889127985 5:0.24030155936964714 6:0.8080778221819038 7:0.05395496402881128 8:-0.3045148076729973 9:-0.6134406357458853 10:0.7447268183581948 +-11.328415936777782 1:-0.10183127796258096 2:0.5689039487721601 3:-0.07015335898840225 4:0.23254189629731292 5:-0.3226974656715038 6:0.2859450214054784 7:-0.4916677058012495 8:-0.27564895614732055 9:-0.9416483232894219 10:-0.7472248333434015 +8.719164753818454 1:-0.8231424386390782 2:-0.03953537069863633 3:-0.3271580541537027 4:0.892192314973022 5:-0.6759017192358232 6:-0.419591686354591 7:-0.23967385135363606 8:0.936992531568956 9:-0.12946409158671512 10:-0.9082863469271643 +22.31738046492344 1:0.37030851555335365 2:-0.06654751559177563 3:-0.5759425437665169 4:0.9179952251152963 5:0.8628921839116359 6:0.8421952184405965 7:0.9625804174561126 8:-0.03075332253237728 9:0.12227386374957994 10:-0.6243390357793757 +-1.189108450798179 1:0.5681776913545951 2:0.46049028271139436 3:-0.366463711956754 4:0.025856437432560275 5:0.7547565372954261 6:0.5506193192167212 7:-0.6279807084274867 8:-0.38698884324386107 9:-0.9885778854008227 10:0.7814740172261654 +2.8767042393531965 1:-0.6841229745503388 2:0.6252203895646273 3:-0.6737644654353572 4:-0.7321040107741059 5:0.3162570540986238 6:0.6211089085315002 7:-0.33984617437403464 8:0.1227089818682312 9:0.04586594421613177 10:-0.4679977358965799 +2.783332151730615 1:-0.39148258540779013 2:-0.3037233649803406 3:0.7955133548911926 4:-0.1729544208044842 5:-0.18247049275020033 6:-0.1315085429729259 7:-4.447133918370483E-4 8:-0.805837119503338 9:0.11574866650006688 10:0.8517519041042676 +-8.99205564094827 1:-0.45501536967706535 2:-0.35829694693457914 3:0.775695048377375 4:-0.25331195582275745 5:0.15524612858817055 6:0.7400717904631442 7:0.8382485596668376 8:-0.5619009369436814 9:0.4386801597659249 10:0.09960232210246622 +-9.808386702564658 1:-0.987404834666963 2:-0.6732308850750186 3:0.5528285725528492 4:-0.8796302275267409 5:0.30705569958232193 6:0.8635312232105203 7:-0.14033675947074187 8:0.5516086773506235 9:-0.7487899106678442 10:0.8851518933134919 +4.948281656077033 1:0.4331269064492329 2:0.4628446087354616 3:0.33730748244242537 4:0.3473124014683382 5:-0.1707966473106064 6:0.8558057784524846 7:0.1390312032172829 8:-0.7918343112673001 9:-0.85993782695915 10:0.33563174747577107 +10.791261476321019 1:-0.5417345768902055 2:-0.06334901799780424 3:0.027652223245870466 4:-0.9881487640651161 5:-0.19441123027957707 6:0.40295156581142355 7:-0.8315553696517317 8:0.11405283165483926 9:0.5377980570161418 10:-0.24581620554740824 +-0.7287230169119936 1:0.33985587202063283 2:0.6841261099887705 3:-0.9441564997438197 4:0.28660913255058906 5:-0.7597915572726905 6:-0.8535957517473378 7:0.609134673753593 8:0.29636368731717977 9:0.05791523580926916 10:0.5589907965230858 +-26.805483428483072 1:0.4572552704218824 2:-0.576096954000229 3:-0.20809839485012915 4:0.9140086345619809 5:-0.5922981637492224 6:-0.8969369345510854 7:0.3741080343476908 8:-0.01854004246308416 9:0.07834089512221243 10:0.3838413057880994 +-16.71909683360509 1:-0.24375714099465773 2:-0.11915875769929496 3:-0.3741442802364221 4:-0.3812947578178094 5:-0.7032156297055756 6:-0.18339122712542388 7:-0.8634662520461855 8:-0.714561692659166 9:0.020558676493369177 10:0.22804428969949986 +-8.822357870425154 1:0.39332200105884363 2:0.5652370435795515 3:0.6220479966351453 4:-0.018976695481651484 5:-0.6868425195058918 6:0.2029750380170401 7:-0.5550873767310935 8:0.16864133648532342 9:-0.008843355054633628 10:0.6472547984399621 +0.36392761004065594 1:-0.9059630492963144 2:-0.41039282402227384 3:-0.006673269562094131 4:-0.4989314017618798 5:-0.17726034513032318 6:0.037764439388023874 7:0.30703957185016595 8:-0.09040426404909185 9:0.38661451965066274 10:0.1630571642147851 +7.415902871490132 1:0.188586850708651 2:-0.33013604761672566 3:0.6667976416858177 4:0.8537064956198137 5:0.03971370422819254 6:-0.43229195778759966 7:-0.9607154505216515 8:0.8413204878098277 9:0.40010565279599897 10:0.7306602852367441 +-4.129456164370826 1:-0.7967510984807558 2:0.545111159425699 3:0.16038228447433012 4:0.6311115528116698 5:-0.01985759480036542 6:-0.9516543115476572 7:0.18022912194075458 8:-0.2177157123823752 9:-0.5433158910016767 10:-0.4603867691069983 +-9.211066571082247 1:-0.3611235296125135 2:0.1402619601475985 3:-0.23132525512647795 4:0.5534401725834837 5:-0.34978585787763206 6:-0.24147682088922773 7:0.8089009287617064 8:-0.09075864922490862 9:-0.05759391404550773 10:0.3371306765964468 +6.52392916461972 1:0.19122050285976044 2:-0.625453376800498 3:-0.26804961781489856 4:0.9669297468261109 5:0.9142504122291741 6:0.7678963028488108 7:-0.6852943621882759 8:0.5898129788981794 9:-0.6580947533327339 10:0.46875109532259396 +-12.46765638103286 1:0.35148385951742633 2:-0.5206883134357769 3:0.35436280451876345 4:-0.8837833467474128 5:0.3433887284719144 6:0.3914771858025621 7:-0.17813796710416252 8:0.6553344538056296 9:0.3721548243590813 10:0.9442185832979726 +-4.937258492902948 1:0.9150659354384785 2:-0.17085510578573548 3:0.8233227233543232 4:0.2539669132090434 5:0.18955049451212935 6:-0.2833188558310358 7:-0.48483747414616496 8:0.8917378487725669 9:-0.13169122011498646 10:0.9815059855284158 +-0.5233425797210233 1:0.4238363705720569 2:-0.18363058784066522 3:0.2949874786744968 4:0.12235592695567354 5:-0.9746310186182559 6:-0.8990867637441311 7:-0.8580982328464586 8:-0.7930887027205957 9:0.16757307988090275 10:0.988861929608575 +-11.904986902675114 1:-0.3692990475534952 2:0.32166293883244323 3:0.3401547722249436 4:0.10009747375878408 5:0.7598877208920192 6:0.2853003389082669 7:0.22880221701675074 8:0.4521491122351502 9:0.33222018268933895 10:-0.9500018867461919 +8.324969054805921 1:-0.48086111720736513 2:0.3705524122401185 3:0.43635448766342133 4:0.6544321903349255 5:0.059000747296945155 6:0.3328036763371236 7:0.9609146376298034 8:0.5943082361322021 9:-0.3074246170581105 10:-0.6763916655761453 +0.21701641918233017 1:-0.29449708766806304 2:0.040640346437143426 3:-0.6524819533513639 4:0.37482287233702394 5:-0.29800608396043216 6:-0.537030944860492 7:0.2862394027536084 8:-0.3783043133672048 9:-0.5292179323972728 10:-0.09583783955916791 +-6.84977373580439 1:0.825136109631339 2:-0.5722868691442817 3:0.11048134523744757 4:-0.5946054293068455 5:0.28061485657354823 6:0.9135611623885838 7:0.35590421873954603 8:0.8943562249941011 9:0.4183378981109729 10:0.5714160298247304 +-11.039347808253828 1:-0.9620263418414967 2:0.22669065740934724 3:-0.7378036492234086 4:-0.4460191511609126 5:-0.2594476006347024 6:-0.989879976130936 7:0.762096015449097 8:0.6983868222083149 9:0.8729993459982626 10:0.3426647417451305 +-5.882860061103163 1:0.5247178959769465 2:-0.6217169944869176 3:-0.13640714414758315 4:0.6608201052790283 5:0.5789945243704264 6:-0.12686057623612612 7:0.7277882307863026 8:-0.47949544949858236 9:0.9781208432412936 10:-0.8980068284379361 +23.52945433069272 1:-0.12339549394875426 2:-0.6769524283089239 3:0.9324962870874394 4:0.28956947294105206 5:-0.2957355479338608 6:0.7504385350771912 7:-0.8769262306643106 8:0.41591311300668155 9:-0.7694611231426498 10:0.9885110924181837 +19.043184423383824 1:-0.13783178628851878 2:-0.853631844645959 3:-0.12792415583066052 4:0.6936898387576049 5:0.8488563282318959 6:-0.6530521292304581 7:0.27832187660440666 8:0.09838048719062442 9:-0.5913230087557231 10:0.260839433107553 +6.83105883806984 1:-0.9085282656519695 2:0.65203708247844 3:-0.687580071985604 4:-0.045008726377529173 5:0.4762107922777967 6:0.15939259525248506 7:-0.46363191848939334 8:-0.25856682230410266 9:0.313842004143269 10:0.5042938214484851 +-9.409197719620593 1:-0.34356198962701945 2:-0.06381545064099514 3:-0.9332814619122063 4:-0.2629675367181199 5:-0.03876014002851913 6:-0.4606936151803749 7:0.49272969757318563 8:0.5550196351479111 9:-0.1758425343811718 10:0.20285868144226837 +-1.3101852978323116 1:-0.3740821549570985 2:-0.9788976137554464 3:-0.6078739734947245 4:-0.8007745980271539 5:0.7381298546055934 6:0.7407750458109124 7:-0.7711351008178868 8:-0.9895256155202141 9:0.35793767138197174 10:0.6589909255086295 +0.5180809608973377 1:0.19289850282287446 2:0.6301214514538145 3:-0.15311307199521518 4:-0.8607670552113709 5:-0.46422067276745316 6:-0.29812862604449464 7:0.519464836430044 8:-0.9480450997338103 9:0.973503038633444 10:-0.7843880226794626 +1.9947872601406775 1:-0.15799682110486057 2:0.22645891561571352 3:0.3141842574216682 4:-0.36086019480721676 5:-0.1429373936064291 6:0.8097261636650581 7:0.11764088861630029 8:-0.9151998265501957 9:0.6536711690904891 10:-0.17232697113157425 +12.352290000973428 1:0.8176113135335772 2:0.39342616792621987 3:0.44011948797971234 4:-0.4412435869837865 5:-0.24509203724837314 6:0.8636655043434542 7:-0.4251583124505798 8:0.2068056615503988 9:-0.3501114760443049 10:-0.23701353324739483 +-2.891643319177732 1:0.7722403010820704 2:0.7994121584045861 3:0.18520464815273208 4:0.7273575609391227 5:-0.3758589216283552 6:-0.7598404862373955 7:0.5748649410179301 8:0.6897988099260968 9:0.5638920860629713 10:-0.992567809902162 +4.803737144054077 1:-0.7367711178556622 2:0.07370548192399351 3:-0.5510509754264419 4:0.11949095653894504 5:-0.7723751845800411 6:0.6450480728551136 7:-0.9508825019800493 8:-0.3250395411575804 9:-0.24913562167143777 10:-0.3617439870343031 +5.051689886526102 1:-0.09854955786627007 2:0.5298224514703289 3:-0.014996634675966236 4:-0.4462048687049027 5:0.22912790083984547 6:-0.513533454471272 7:0.1452771069237353 8:0.371152210841464 9:0.9204732090987018 10:0.7472990716905279 +3.8591142298280476 1:0.7532169023970261 2:0.8291433156934658 3:0.9255891263525324 4:0.3248663809949248 5:0.9905320652281553 6:-0.10383453745167626 7:0.8519246838852608 8:0.6024015353989258 9:-0.06958036249881938 10:0.5862142389541998 +11.30005914221598 1:0.026411858067972194 2:-0.6968445330429607 3:-0.8194566946165238 4:-0.12780659247925996 5:0.8406393783194903 6:-0.24617182945415128 7:0.30199973460219853 8:0.6062457235841974 9:-0.19314055910416927 10:-0.48313233883372964 +-10.288657252388708 1:-0.7388306404020344 2:0.07753617971873439 3:-0.5735498713988352 4:0.2183581175474576 5:-0.873572721679176 6:-0.8788755575751708 7:0.7087858362905568 8:0.7126712562404713 9:-0.7607334319316799 10:-0.4627367552114916 +4.895250842405817 1:0.9772954128558484 2:0.6020087399988574 3:0.16946626176056134 4:-0.011334492807484997 5:-0.5391845039589362 6:-0.4315843612118535 7:0.9065130011032458 8:-0.4860160207844919 9:0.0921755607946162 10:-0.022200673265013515 +1.0479421939727227 1:-0.055436367433274514 2:-0.6710483362647659 3:0.9222786043047919 4:-0.22005981623386184 5:-0.8141845044113469 6:-0.31766631447334226 7:0.6067696845798944 8:-0.1445661385071555 9:0.9172271611227454 10:-0.8079554780561127 +-9.754451457291598 1:0.533713237587885 2:0.6499588942067549 3:-0.49188790503368285 4:-0.6925119436487435 5:0.3345265979579788 6:-0.8117849521672496 7:0.9312055115656304 8:0.3273803451149724 9:0.7567478475677727 10:-0.6256676928549367 +5.869027126482974 1:0.7273823383600513 2:-0.2519813990388706 3:-0.8239584025397881 4:-0.13749750031735974 5:0.6142824732416132 6:0.6251630800232315 7:-0.6138240706157267 8:0.7210396245391326 9:-0.41832155201953714 10:-0.8965988320689853 +9.14234252751227 1:0.7295320896113133 2:0.6150271212503227 3:-0.9785024737101733 4:0.30006672036705506 5:0.11703528191771406 6:0.2971639460196238 7:-0.7920108995168815 8:0.32649036066184567 9:0.03522428067355543 10:-0.1766251898148803 +-5.643698771141404 1:0.27360638280623983 2:-0.6124401810442446 3:0.24950528730210886 4:0.09920211684887548 5:0.7187490549286091 6:0.6212724115415782 7:0.5864634211269566 8:0.114951165007104 9:0.44859258949094283 10:-0.3768352371578665 +12.781643819428492 1:0.9144335582094396 2:-0.4579872615218674 3:-0.6521934534632468 4:0.4462086111316512 5:0.240360283350179 6:0.23974046479581124 7:0.4840439971437822 8:-0.7250363120037027 9:-0.29769496257362094 10:-0.3382859512018359 +8.393556738722923 1:-0.8263387132502396 2:0.9434824094966923 3:0.1607861709872136 4:0.15217100448798782 5:-0.6517945935711484 6:-3.354731073326178E-4 7:0.07846631386981562 8:0.687844846942889 9:0.9277854407325892 10:-0.8855380268588307 +-15.348871155379253 1:-0.5734707274250155 2:-0.2526008551945753 3:0.23752094195309925 4:-0.7074613963298721 5:0.4674168537545218 6:-0.3198997855552628 7:-0.10415974108745596 8:0.5616912699671224 9:0.43742425558560694 10:0.19732530755184596 +13.138260063721448 1:-0.9415220143797984 2:0.6015431361268124 3:0.38898046240229545 4:-0.5750448371021175 5:-0.5803995196333898 6:0.11772198725731342 7:0.7512685244060366 8:-0.6683465740662857 9:0.9515652825318053 10:-0.32405935964523547 +-26.736207182601724 1:-0.47083104147202404 2:0.28748860067800597 3:0.007399318769021113 4:-0.8189013750589702 5:-0.5156633937248272 6:-0.9906928746525896 7:-0.8848419810272337 8:0.2197280161306785 9:0.12855082514870197 10:-0.7862803985146845 +-20.212077258958672 1:0.5609065808412279 2:-0.9201904391147984 3:0.908305865183735 4:0.9255146658282842 5:0.6871419344095282 6:0.4201876217923466 7:-0.42906289792612684 8:0.5787691868233418 9:0.7260522064761288 10:0.28251641556690554 +-0.44652227528840105 1:0.37640618494870504 2:-0.20012451052963542 3:0.9420894309510319 4:0.4218728633972739 5:0.5551974480349577 6:0.07615991810462619 7:-0.12409220462011294 8:-0.22212591926375946 9:0.21160498862483723 10:-0.6092792830633924 +-1.9481059746438067 1:-0.43820030250217457 2:-0.6836588417639442 3:0.733018205278934 4:-0.6564348753121718 5:0.7333385435136448 6:-0.5577457688360317 7:-0.31035811050608975 8:-0.7189201447768139 9:-0.7629842028723994 10:0.7179459779331092 +1.1951162998609508 1:0.19541555859727744 2:-0.4796785506546435 3:0.14123852670749248 4:0.7161847585887089 5:-0.2502765085719578 6:0.8815667909545981 7:-0.6418691905513725 8:0.49600147195728783 9:-0.3091837674381053 10:0.4320162841463153 +-8.99125390483227 1:-0.01183888602092864 2:-0.5901829024081027 3:-0.4343074406380647 4:-0.40450313056290166 5:0.05269590196351448 6:0.733631212862198 7:0.9575176715505025 8:0.5974628692830348 9:-0.20284241796038271 10:0.9577348510907686 +-7.955533026930219 1:0.6104830760481679 2:0.5915483572646505 3:0.3275427350991458 4:0.48361434056132424 5:-0.9466590639056058 6:-0.24662428438925743 7:0.9856361456534972 8:0.9434155212648045 9:0.3466736921968707 10:0.12927980558284102 +-12.500773785355054 1:0.5733321361720694 2:0.39154119830075085 3:-0.9347116355607772 4:0.0920586614926524 5:-0.6959457183810456 6:0.2136579936466858 7:0.17595268059814395 8:0.8828168055200465 9:0.18934277314853398 10:0.7565908584660754 +-11.43180236554046 1:0.082018621904135 2:0.9074181204118958 3:0.46125595008850273 4:0.40328845936169966 5:0.7803064691948824 6:0.20802011482729377 7:-0.41368899649077284 8:-0.8997565495498339 9:-0.1880483213318005 10:-0.15538597634233264 +-5.055293333055445 1:0.4442675297698402 2:0.19045719972922193 3:0.4877438951288897 4:0.7984474402420494 5:0.3251350777349489 6:-0.18676050499673869 7:-0.2701840041572374 8:0.4486609996458524 9:0.5403637876036615 10:-0.8971614841211264 +1.0276485382241776 1:0.7953696703382547 2:-0.3245779681908927 3:-0.3507435626548021 4:0.9510986059491036 5:-0.8655491074076527 6:0.20729233888498677 7:-0.43078300089533594 8:0.19504657032168216 9:-0.3173814102187291 10:-0.042479969052890754 +9.690201571311908 1:0.16852987139559206 2:-0.2514893273405625 3:-0.9993240281686275 4:-0.2166013247997891 5:0.33294165754921234 6:-0.5824203831560628 7:-0.15253642946648616 8:0.3547892367555441 9:-0.047604356104869794 10:0.9229112136183077 +2.2591036039970347 1:-0.9919593184325572 2:0.6323551392201245 3:-0.20815293136790447 4:-0.002395046469600759 5:-0.5015903362190326 6:-0.16698803749234048 7:0.7901657583805675 8:0.33755402936964973 9:-0.3707337678548108 10:0.6995480653730146 +1.5130881908855742 1:0.973710432688613 2:0.6518972988019702 3:-0.16491318496856833 4:-0.6066757853095415 5:0.8762371591845273 6:-0.9056066630820714 7:-0.3388079327070965 8:0.3934146060660142 9:-0.8756168865642253 10:0.9522427911640303 +4.023618949132531 1:-0.14974626191548301 2:-0.5874962377709136 3:0.6780439909311404 4:-0.37291203746764356 5:0.08104034602232169 6:-0.4706923395029945 7:-0.8924577368048239 8:-0.3363784341297067 9:-0.4139746050396018 10:-0.5107600309932907 +-2.8674162893420965 1:-0.7554383289076523 2:-0.1355597928418868 3:-0.3891904246986413 4:0.43949832438341785 5:-0.43859957095446833 6:0.37548094528561093 7:-0.5228633291549518 8:0.24169710795100352 9:0.7131753590746546 10:0.03458176767001042 +4.661164232198611 1:-0.12738868751385546 2:0.9446285809821182 3:-0.17981416859193433 4:-0.7535879975625193 5:-0.08594548726529161 6:-0.9983154486609989 7:-0.7272748852665216 8:-0.8197811039616518 9:0.5177610923333253 10:-0.6180731281817853 +-0.12347625601866746 1:0.10820547757674692 2:0.1825421454873002 3:-0.3412486258429426 4:-0.14925445930975534 5:-0.6594599831395103 6:0.9552502376248448 7:-0.7875626067291472 8:0.3854984181307912 9:0.014303876202374832 10:-0.7300443667550689 +14.546296184422973 1:0.2459523985646046 2:0.9434777073825811 3:0.2112745925235362 4:0.7730688005214974 5:-0.13727994893203732 6:0.6140037510172511 7:0.7545298281668846 8:0.7814551909982614 9:0.0026683642139069264 10:0.5633973602849358 +-19.66731861537172 1:0.9353590082406811 2:0.8768609458072838 3:0.9618210554140587 4:0.12103715737151921 5:-0.7691766106953688 6:-0.4220229608873225 7:-0.18117247651928658 8:-0.14333978019692784 9:-0.31512358142857066 10:0.4022153556528465 +18.84119697288412 1:0.4423204637505467 2:-0.4364821709544735 3:0.3935363893778452 4:-0.7750286735195999 5:-0.6981814766625978 6:0.6889512553826111 7:0.3646791168217727 8:0.0023536025493677837 9:-0.08378048150085249 10:-0.05659381771155503 +17.40329212914592 1:0.9155980216177384 2:-0.35593866074295355 3:0.44775710780914824 4:-0.42914421567532357 5:-0.2734430718503955 6:-0.8937042912745483 7:-0.3143761936611371 8:0.07805814979426184 9:-0.31386151509289784 10:0.6202932236456253 +-19.402336030214553 1:0.462288625222409 2:-0.902975525942725 3:0.7442695642729447 4:0.3802724233363486 5:0.4068685903786069 6:-0.5054707879424198 7:-0.8686166000900748 8:-0.014710838968344575 9:-0.1362606460134499 10:0.8444452252816472 +-3.855123203007599 1:0.5072557393175969 2:0.4626973233672753 3:-0.20910077161652119 4:0.9431415515135266 5:-0.1293690767585638 6:-0.2033835058111637 7:0.501429131658198 8:0.175133281735671 9:-0.6091682952201736 10:0.543010689352589 +1.493768355655548 1:-0.7772812666041105 2:-0.7743738591348672 3:-0.2848754060915175 4:0.3336846848765145 5:0.6219572132443736 6:-0.11144657683793624 7:0.7606913325884337 8:0.8547085151723017 9:-0.31728444617771134 10:-0.4668474022688931 +-17.803626188664516 1:0.5176340000264179 2:0.23048377874011128 3:0.6162746928601832 4:0.16908590014785418 5:0.9695207469685181 6:-0.34713218673384705 7:0.8526833760069625 8:0.9895592279649763 9:0.8805561957342884 10:-0.43452438291417894 +1.4060200157931342 1:-0.41964471941333525 2:0.7738486114171979 3:-0.0964606192284374 4:-0.25351781452566025 5:-0.21065389913054244 6:-0.40490416354122916 7:-0.7696501777959646 8:-0.7710488116813146 9:-0.6777228721053572 10:-0.09381158095961428 +-17.026492264209548 1:0.8367805314799452 2:0.1559190443625338 3:0.048200110551483544 4:-0.7340083467235765 5:0.2661150265782781 6:0.3881661781792165 7:0.9485287302765621 8:0.7201540574376382 9:0.8509234862656003 10:0.9658114866648093 +8.729450606651499 1:0.6404862166906327 2:0.16516090922657822 3:0.29013117743588057 4:0.37056732180613317 5:-0.3376494575302882 6:0.9012625630650577 7:-0.42150978319487 8:-0.05630249989686087 9:0.706104255632954 10:0.01935884085365225 +-5.516822117602276 1:-0.5718348423045241 2:-0.2145777722920088 3:-0.09307467998835195 4:-0.7311274103678378 5:0.5272184003067053 6:-0.00528176138162495 7:0.2852826178935919 8:0.6180999884045897 9:-0.7526372151008776 10:0.20416472532830543 +13.001541259752251 1:-0.5137703877272299 2:-0.15452359837207896 3:-0.25657600903152744 4:-0.9773110735601165 5:0.0718147980090178 6:0.18965211809311744 7:0.7795354990363292 8:0.21976898743223638 9:-0.20364089221752524 10:0.33822332985943304 +18.443388694564348 1:-0.9278344397401963 2:0.2678538727090136 3:-0.46932389854374734 4:0.8494176173177825 5:0.45765527018197694 6:0.20546395745879287 7:-0.199860294349123 8:0.47798730134403256 9:-0.2279771893187592 10:-0.30836118564314274 +8.952089112152663 1:-0.7371671220953286 2:0.8160149639986789 3:-0.026630089188139028 4:0.5931015267817183 5:-0.12216243475451294 6:0.161290795125286 7:0.7423016751095652 8:-0.5212872902985852 9:5.606147011660845E-5 10:-0.409626733921443 +-3.7062463981908027 1:0.5633514321449928 2:0.9914900963311462 3:0.1867799930236702 4:-0.15960235736142847 5:0.1204791067384241 6:-0.7733281422620872 7:-0.887447048141158 8:0.7931515335800692 9:0.732289882696125 10:-0.034992898370363124 +-10.58331129986813 1:0.6627003739767989 2:0.10688718810947728 3:-0.49230090744757216 4:0.8936580036513948 5:0.012227929286241057 6:-0.1442038886014838 7:0.9203452040795139 8:-0.20719832624131262 9:0.29561869366253335 10:-0.08597725084864649 +9.818996211259908 1:0.580133516885796 2:0.07422424429848573 3:0.33438634998226924 4:0.26054797992533696 5:-0.8771304726537796 6:-0.9710990591964794 7:-0.1869287393875041 8:-0.6167738073093247 9:0.34401921428837245 10:0.6737600514607418 +-11.87816749996684 1:-0.7193071334885193 2:0.5247127705364141 3:-0.02978727198197606 4:0.18353223007701058 5:0.40350110058596944 6:-0.36002841871228686 7:-0.20781535546501528 8:0.5517883176456557 9:-0.9938027872744732 10:0.6245061418135955 +-12.198096564661412 1:0.27542314155961156 2:0.3459734388741733 3:-0.38737776987446937 4:0.6244101669171684 5:-0.7801218302490938 6:0.20444733666197523 7:-0.5667599464182904 8:-0.9462131580071358 9:0.5576565405741785 10:-0.9307557040059242 +-3.6610413123521357 1:0.045569951437504086 2:0.32203961277046145 3:-0.04228927426053675 4:-0.9435304938416831 5:0.3750509710699601 6:0.21298970117620142 7:0.5491054691791977 8:0.33695088608872203 9:-0.9923500858828505 10:-0.6402707119893463 +3.782742149409224 1:0.7795250611996376 2:0.43296979846218275 3:-0.6481485005937841 4:0.3235717281667645 5:-0.8067382770768907 6:-0.06740397503468509 7:-0.2835017205434338 8:-0.5875853498478532 9:-0.25699561837680585 10:0.7813561594373908 +-17.065399625876015 1:-0.01772446594568744 2:0.563282914714494 3:0.14232420381013955 4:0.031667902604941345 5:-0.7815348482900619 6:0.3657733497576803 7:0.7208326162626688 8:-0.7863253120180662 9:0.3329194167867533 10:0.6175752945608013 +16.23248797654815 1:0.2615647748812251 2:-0.6631801348538622 3:0.6420349382574477 4:-0.31980528388089846 5:0.38021930887251365 6:-0.060298437830818896 7:-0.8911652782989568 8:0.3424617259589986 9:-0.8515350749364614 10:-0.42354709676980207 +-5.015963911416578 1:-0.07890564237014686 2:-0.09864377281008885 3:-0.13139943914680408 4:0.6610949669857866 5:0.06777579108221987 6:-0.26586245727222835 7:0.17443498956808612 8:-0.3129854922817781 9:-0.37913757211269505 10:0.7627186373372121 +22.647750304177556 1:-0.03666997412165163 2:0.49691867674483814 3:-0.45898559472166967 4:-0.09932248891016404 5:0.05692910907689508 6:-0.5006743461081364 7:0.9992936758550379 8:0.8252525466172065 9:0.9431711015127009 10:-0.4891497061921315 +-3.731112242951253 1:0.44353490207818513 2:0.23112032838224117 3:0.4697682541445527 4:-0.7507514828346664 5:-0.06323257550543837 6:0.0997091431243109 7:0.9394036761509628 8:0.4103869738859962 9:0.6564209227640914 10:-0.5427466755921158 +0.6761872737225261 1:-0.30051626190360503 2:-0.26699232020158803 3:0.8668758741279379 4:-0.40325291744583347 5:-0.9756425738484267 6:-0.5116398654634617 7:0.16424789009043073 8:0.8034099442414044 9:0.8554935001446193 10:0.42747702930667497 +8.449247195197387 1:-0.6588765973399024 2:0.2502285196526799 3:-0.20481547024283087 4:0.3770725284683252 5:-0.169707887761277 6:-0.0804075502584003 7:-0.3580757176408007 8:-0.6042549664471129 9:0.360349278976142 10:0.15899650901110962 +27.111027963108548 1:0.7106841652047162 2:0.6853699382312817 3:-0.8076297545289823 4:0.7932321056591545 5:-0.8011085095234463 6:-0.7017292726737878 7:0.10568649778064154 8:-0.40755358264969255 9:-0.061008981132773865 10:0.08895972651409556 +27.78383192005107 1:-0.8378790218922778 2:-0.6651002504721837 3:0.021049638665430415 4:0.32994334871293196 5:-0.7981304887988308 6:-0.2947962117284566 7:0.9739408711845776 8:0.9442893181893954 9:0.010541491359981059 10:0.8332791453382604 +15.700710963871254 1:-0.538773982400854 2:-0.5966426806845984 3:0.14570292467314627 4:-0.5937791901212952 5:0.7779251136963325 6:0.9962962075803357 7:-0.4774083823748394 8:-0.02528476957876369 9:-0.17305036341254398 10:-0.6013841506503688 +-12.558575788856189 1:0.03250364930617211 2:-0.6723950859659307 3:0.7090474884514901 4:0.25034305882632735 5:0.7036774024093582 6:0.622650236684523 7:0.5776881238206741 8:0.7999754726258337 9:0.21332972563833508 10:0.33849062947231645 +6.2776776518215955 1:-0.009605588630256623 2:0.5786496865369053 3:0.9208276908400748 4:-0.9477397424337148 5:0.6306053656362194 6:0.5396434662389846 7:-0.9841930450269964 8:0.5492682920407823 9:-0.020767248025529206 10:-0.8684655435686472 +6.424586997399564 1:0.861374923392324 2:0.8356037964367176 3:-0.7173479824827564 4:-0.6309584820438245 5:0.16136758138471285 6:-0.7485184163431866 7:-0.006053583829132236 8:-0.8762221084691306 9:0.19195377669247726 10:0.07259634302552964 +-9.64772485466405 1:0.7568015336230662 2:-0.4221524485756756 3:0.011711847664269248 4:0.7387065048724242 5:-0.04347512566745104 6:0.06642100869974654 7:-0.6993705848315939 8:0.16312217088045422 9:-0.11975577990989916 10:-0.6188717473788392 +3.8183706502283647 1:-0.7226937936463145 2:-0.5462756960199258 3:-0.39158419906610664 4:0.014310440945434433 5:-0.9950315917350652 6:-0.1844037449550875 7:0.9023517651879036 8:0.7948752060508435 9:-0.6792702010973877 10:0.40730074403235617 +1.1585019476700562 1:0.5575546848694 2:0.8997032130006739 3:0.6088643323129037 4:0.4872893656051758 5:-0.03977520372748922 6:0.3202565433572042 7:-0.31231768645537206 8:-0.6861153669592381 9:-0.08561643820383291 10:0.522243657731251 +-8.18651039877047 1:-0.809069379967462 2:-0.04827229852445103 3:0.19963602092982624 4:0.2568971171641006 5:-0.0015346733366310428 6:-0.6104625526166494 7:0.7746715041233412 8:-0.7343750018341593 9:-0.49272635466510106 10:-0.8115191199688623 +-3.377690136019927 1:-0.9408187510685164 2:0.9654993263332854 3:-0.16725010447984268 4:0.2574069587853294 5:-0.6930506968932861 6:0.11124762075550176 7:0.39145805505914866 8:0.2906495128462767 9:-0.27454907309824916 10:0.9001175309434777 +12.692571815413245 1:0.7404426710258791 2:0.9060576634778448 3:0.7023712021897308 4:-0.9808126157768493 5:0.03447666475715194 6:-0.4146339211599541 7:-0.7329651749553896 8:-0.2696019807317358 9:-0.9885367164723897 10:-0.8540304023043486 +2.5111054050889354 1:0.7448154454968356 2:-0.7532143233138027 3:-0.9724617436335079 4:0.662620399592766 5:0.45517204589358307 6:0.37409736074838684 7:0.337245076577648 8:0.50951903847353 9:0.2590369923587328 10:-0.3248257475117191 +-8.300340493749207 1:0.5504850435404609 2:0.5077232940244447 3:0.778859307357816 4:0.2601916883813373 5:-0.0032275666062382413 6:0.039752927221862855 7:0.19468432568826755 8:-0.2859531554546477 9:-0.4113477962970582 10:0.43272011953041667 +5.904938653193952 1:0.6622293273002955 2:0.6428891633785236 3:0.6999663090423285 4:0.9132698742913088 5:-0.3960072336866507 6:-0.14500922264286054 7:-0.4390171033743564 8:0.002067106212897185 9:-0.6079874251539117 10:-0.7131416109696531 +5.004048239623824 1:0.7212309895357449 2:0.3425199843383353 3:-0.7290323633040705 4:-0.5563097960397918 5:-0.7577898297822001 6:0.647883070472203 7:-0.23710559062843073 8:0.34398507133293954 9:-0.5440251617348038 10:-0.2971638032112218 +6.21255598077158 1:0.2498685983586959 2:-0.2586857335205359 3:-0.6380810501916263 4:0.17008841621855852 5:0.9485802018202867 6:-0.2580306792121272 7:0.032916516140567786 8:0.32950951532163675 9:-0.9291915084526683 10:0.8454021164786922 +-3.741044592262687 1:0.763300390779396 2:-0.1832552896771813 3:-0.39361907876758573 4:0.9050768615040607 5:-0.8850093869496836 6:0.9302208653737598 7:-0.12972094056755412 8:-0.459442486378308 9:0.5044112394875107 10:0.1399067554681861 +7.378402183384303 1:-0.27686808475610114 2:0.12735524561214606 3:0.5216635958678004 4:-0.9418584785460469 5:0.20441570818728771 6:-0.35073421178920583 7:0.7847501694079704 8:0.3222999552829353 9:0.21025696511089764 10:-0.5813710201294744 +-7.1500991588127265 1:-0.1945259148773102 2:-0.4089845159829022 3:-0.1971859124232922 4:0.9531447983295496 5:0.07996455700202221 6:0.17013529724757648 7:-0.2442095218739362 8:-0.8564146371721229 9:-0.5843910532907555 10:-0.33846471424918767 +-4.288417758202577 1:0.020710986120182184 2:-0.7450564238727908 3:0.3674992023059285 4:0.46737461414601555 5:0.9411702705113052 6:-0.7257365059912877 7:0.5813280037560231 8:-0.01567531846894843 9:0.24734195293533467 10:0.6516001002566887 +5.916426037500391 1:0.8260000862135342 2:-0.11324162495165968 3:0.13061304369435334 4:0.5762591624576425 5:0.548049763999644 6:-0.9751599851764361 7:0.02828821483057764 8:-0.4113286027346803 9:0.8912856976307486 10:-0.8470910204808244 +2.431004294471012 1:0.14088576701299083 2:-0.45104190898994734 3:0.29891134031619115 4:0.955503074037666 5:0.15962522624750242 6:0.7664481093046553 7:0.051697815479792686 8:-0.3471787155014081 9:-0.8007151537631465 10:-0.5598899500902301 +-16.08565904102149 1:0.3946137229565083 2:0.8443779319638349 3:0.5116855547320893 4:-0.5319339991982652 5:0.26564506849312797 6:0.18905397829944448 7:0.1976357098053687 8:0.15505612242632538 9:-0.935633748308776 10:-0.9782957013204887 +18.058440348477184 1:0.8402487524597533 2:-0.6200725197687718 3:-0.6158487677192792 4:0.0709328308135515 5:0.7501256905495493 6:0.38092209802839583 7:-0.8192579128383128 8:-0.9304002828581583 9:-0.6570300818845025 10:-0.5252554781538985 +-1.0026720160736349 1:0.46122079684901474 2:-0.7609201036934166 3:-0.9372178059537293 4:-0.25391036498391006 5:-0.7487429157699828 6:0.38024314675291637 7:0.21886059803198576 8:0.027516853267765207 9:0.33483464322377765 10:0.618580130027746 +-2.6688695419207162 1:-0.8775911623423445 2:-0.6647410420697879 3:0.05948516302547313 4:0.7278526664475804 5:-0.011366224409705028 6:0.33475665968289436 7:-0.6386120399761575 8:0.39609772177595115 9:-0.7872076290319412 10:-0.6195857302948329 +-13.867087895158768 1:-0.9114780602695882 2:0.7997695296649912 3:0.8337252417804881 4:-0.7927267913881113 5:0.6863829853181673 6:0.4162562153517635 7:0.2659922421074139 8:-0.551994669040742 9:-0.6403900338772157 10:-0.8680387717518072 +7.826011095515239 1:-0.2881951904396949 2:-0.19317071325391022 3:-0.06581062483451183 4:-0.6074074436315555 5:-0.9434740067975405 6:0.9426572655575483 7:-0.1812629432036228 8:0.39425575292939863 9:0.5065890539615039 10:0.8969825696966649 +1.4213836206303339 1:0.6996840540120932 2:0.1283999569152492 3:-0.2537375462472613 4:0.24772110606788456 5:0.9040210381745799 6:0.47062010977660207 7:0.9697678931927365 8:-0.9215764371674713 9:-0.27541598110075793 10:0.44277003247067803 +-0.973650798730175 1:-0.2121645467631068 2:-0.6770222508071349 3:-0.5733067523949165 4:0.27979529516037105 5:0.7128588235545461 6:-0.9208763636184307 7:0.14128337151047532 8:-0.002851660400375433 9:0.6943908711123281 10:-0.9201922993121072 +-0.17500848560451965 1:-0.6015070903427717 2:0.7815998200409671 3:-0.9932006200204946 4:-0.3303953411379028 5:-0.3329917860768894 6:-0.2822852019877604 7:0.6834785385197197 8:-0.6458607648553825 9:-0.06171476054995373 10:0.11421513352405444 +-15.310980589416289 1:-0.35290763483001486 2:-0.7263565311032778 3:-0.8688987069582226 4:-0.991098319894185 5:0.7029028082332363 6:-0.20251284356518684 7:-0.10928416773360117 8:0.307764663956116 9:0.6423143148384418 10:-0.15527637175127107 +3.260298266762908 1:-0.7817510582064782 2:0.45336200757318257 3:-0.15365670773321338 4:0.5063951567230205 5:-0.7102867196895872 6:-0.48050036620725955 7:0.9838016675169072 8:0.07854601230194436 9:-0.18953694857147863 10:0.19370072527454107 +3.846123583197846 1:0.6665586449040093 2:-0.2894063530813835 3:0.29965348483445386 4:0.23590344101670313 5:-0.7456743720187828 6:-0.4680876353446175 7:0.8106301610699425 8:0.691280702194663 9:-0.6060141408622055 10:0.34018639920235194 +-10.945919657782932 1:0.7669971723591666 2:0.38702771863552776 3:-0.6664311930513411 4:-0.2817072090916286 5:-0.16955916900934387 6:-0.9425831315444453 7:0.5685476711649924 8:-0.20782258743798265 9:0.015213591474494637 10:0.8183723865760859 +9.820049725467145 1:0.9582163993327679 2:0.7503905881505508 3:0.6255110430336392 4:0.6522701954798096 5:0.09248037700932144 6:-0.2833482854986902 7:-0.9841968940607242 8:-0.9343780716625845 9:-0.605526104070818 10:0.6000165028195326 +11.398715935456183 1:0.6605086903456443 2:0.14675454515266395 3:-0.7880053589830274 4:-0.8570785944515658 5:-0.4317693974151271 6:-0.12244918233307645 7:0.9808241653220866 8:0.5455853515046201 9:0.6870972425676756 10:0.7427686762232875 +-7.846310147695936 1:0.4355817642106965 2:0.7659504362110916 3:-0.3784171977305315 4:-0.5675896574776877 5:-0.20116390539973938 6:0.8775467546326667 7:-0.2824903364469842 8:0.7470660314619617 9:0.8967783051712528 10:0.7133700339519966 +-1.3847391232663768 1:0.3707613476850027 2:0.6931092598460797 3:-0.7701621508103305 4:-0.5679366502518555 5:-0.7234356749703683 6:-0.8059255104944509 7:-0.8307993875388229 8:0.6133975694770035 9:-0.7399749904168824 10:-0.1534990394513953 +16.93981662267873 1:0.6552665678625891 2:0.023248457840923775 3:-0.6850641408327465 4:0.7129790774369389 5:0.04166304042825364 6:-0.7160289667702797 7:-0.4733073680976494 8:0.2720897719417634 9:0.05850741911975099 10:0.34427554125371174 +2.8497179990245116 1:0.6664937514484015 2:0.3343796939204209 3:0.2611910348746209 4:-0.13658810351647 5:-0.5821801257591224 6:0.9854683468621908 7:-0.21396555404689188 8:-0.5923272173716836 9:-0.5674796199927252 10:-0.5681633547764235 +4.981807952389501 1:0.7517426071091595 2:0.7029291090701855 3:0.7126619831046563 4:-0.9982007415355478 5:-0.7743343367502893 6:-0.9048858749551119 7:-0.8243783842398396 8:0.4936163270697016 9:-0.6835495591484724 10:0.8412758607464845 +8.508637575729951 1:0.6837354268578517 2:-0.435346907350056 3:0.6597448795477736 4:0.8870204157376871 5:-0.6938576101541436 6:0.9199495715292882 7:0.33119640706964293 8:-0.6181273221979411 9:0.12929034268333317 10:0.6855150395247027 +14.369378079132883 1:-0.9489372180887643 2:-0.6577177233364067 3:0.543899463531252 4:0.5411152154119976 5:0.43733244485250733 6:0.5927084968109424 7:0.6100068837998656 8:0.9392735722529637 9:-0.9806701698603073 10:0.3984176141500082 +-6.456944198081549 1:0.8380442392342373 2:0.05166133486184443 3:-0.25864153418691704 4:-0.9506672344106888 5:0.5227275493542325 6:-0.03899736644563956 7:0.7660133053649136 8:-0.9375236703284806 9:-0.37213210747743175 10:0.0560768367274771 +-10.041353112580456 1:0.5293717914660876 2:-0.35874932480194044 3:0.14403824250820763 4:-0.4106496629336782 5:-0.794648717231762 6:-0.4369956159772408 7:0.8273613210141495 8:0.9212255384858874 9:0.00409867676727993 10:-0.23796544184855795 +-6.606325361718908 1:0.2765102732490652 2:0.10184669160432525 3:-0.9406443798496789 4:-0.46661976112717896 5:-0.5836573778289609 6:0.1308554421925976 7:0.05232199712543473 8:-0.4965370542771641 9:-0.3695836654343949 10:0.4874427445939513 +-15.359544879832677 1:-0.8253830145927283 2:0.29683545543963885 3:-0.9790356574071053 4:0.33749594518426473 5:-0.449483349548623 6:0.1740013774913005 7:0.5737323257916764 8:0.20159372721320645 9:-0.1812760896634873 10:-0.17652712339895738 +2.1801769966756845 1:0.3664130766917151 2:-0.1929450967547921 3:-0.7834945448457515 4:-0.03806442314852432 5:-0.6167622313628849 6:0.34919852301325394 7:-0.785891329691004 8:-0.5704062599527768 9:0.9846140894872721 10:-0.548571249100203 +-2.7006646885251415 1:-0.48505178676353067 2:0.06347121974094883 3:-0.3704723119141229 4:0.7407080276548548 5:0.06713252857406937 6:-0.2103524488773294 7:-0.9402467715192988 8:-0.8555624501612784 9:0.6244760190429901 10:-0.9038885681517279 +0.2105613019270259 1:-0.17125223509187282 2:-0.23328463772140529 3:-0.6497773470047024 4:0.33111604806115524 5:0.7944287248398398 6:0.5163977380074081 7:-0.025715995643062595 8:0.11762566041047462 9:0.9938658554834845 10:0.5363394203614278 +-0.6433952980357234 1:-0.905126800719938 2:0.5826442985002787 3:-0.8207546276288018 4:-0.0773547002692121 5:-0.6420058913410687 6:-0.9290787206193325 7:0.21829202840889095 8:-0.7752845890678082 9:0.4533233304372326 10:0.5457315861825041 +5.622874731146287 1:0.5486636398086722 2:-0.21867854114956642 3:0.13260110994566032 4:-0.024868470628895967 5:0.9246597814546305 6:0.07490395250443149 7:-0.21327567620097132 8:-0.33970581204395867 9:-0.19408398882121713 10:0.9757334811378136 +-18.27521356600463 1:-0.489685764918109 2:0.6832314342743568 3:0.9115808714640257 4:-4.680515344936964E-4 5:0.03760860984717218 6:0.4344127744883004 7:-0.30019645809377127 8:-0.48339658188341783 9:-0.5488933834939806 10:-0.4735052851773165 +5.518650144654079 1:-0.16881374315243192 2:0.22747702179774354 3:-0.8555270909193926 4:-0.6914231522703247 5:0.03618437407657238 6:-0.8404831131806643 7:0.16378525699004887 8:-0.333895928854854 9:0.23026574917978326 10:0.9409087845740918 +2.5599738684677646 1:-0.24371170373626905 2:-0.1752613047793694 3:-0.7930324885557696 4:0.17288443448968627 5:0.7233942014077801 6:0.47222694561171963 7:0.7878187692414558 8:-0.6520011755878357 9:-0.9952507460157223 10:-0.32951026378415094 +-8.508663400554862 1:0.9194236423060742 2:0.9517284917259223 3:-0.18723709334016392 4:-0.24913001260985546 5:0.8818286401027424 6:0.13661210218384512 7:-0.40792517201812983 8:-0.33132907984544957 9:-0.49137388288628703 10:-0.3273925353006979 +-10.233439586953153 1:0.0960128812383454 2:-0.8611756848964027 3:0.11807312551418647 4:-0.24570750746947145 5:-0.047826307143366886 6:-0.717269426008625 7:-0.2841658181308486 8:-0.31500935950449516 9:0.23183474949267713 10:-0.512986169560546 +-6.3459370724834265 1:0.9537835418930307 2:0.4598546399405288 3:-0.257013655072986 4:-0.29185820894937575 5:-0.6843688281544562 6:0.8343952028925479 7:-0.9656517094615942 8:-0.447440560943553 9:-0.9510349521362857 10:0.5918946980259567 +1.114406550703455 1:-0.5721838436595965 2:0.1201917297381252 3:-0.5253701290141362 4:-0.5874011312890843 5:0.7893580092022578 6:-0.18012813622584134 7:0.4781905737504004 8:-4.6732390143988667E-4 9:-0.7965374182885014 10:-0.8515444146742359 +8.688243146888663 1:0.2245581140502393 2:-0.0697600364101425 3:-0.7661833153629154 4:-0.2289151515902894 5:-0.5643191391300282 6:0.08069861795512168 7:-0.9670317635091523 8:0.14826752863715287 9:0.9325364047311011 10:0.4071178661803092 +14.896035572185347 1:0.20630949870309911 2:-0.5738578325975092 3:0.5664829389128903 4:0.3732752326637825 5:0.04079303403038881 6:-0.6604984910400766 7:0.15136076091734352 8:-0.6244939282579305 9:-0.5236288549540624 10:0.47284992666739023 +4.396558596072123 1:0.5565602414172521 2:0.1444095747909111 3:0.028227502879770272 4:0.38297378287943773 5:-0.26739745457451725 6:-0.708209627997985 7:0.7604483272526881 8:0.8072075261139096 9:0.11460574885028274 10:-0.07669406807610635 +1.7457141275341528 1:0.3668576517164046 2:-0.5352200081463954 3:0.5853385976871426 4:-0.4482551060006992 5:-0.5676795208498786 6:0.8043295590331514 7:-0.02160829797068753 8:0.42281303847010454 9:0.027894531623162466 10:-0.541120112980032 +-15.334767479922341 1:-0.036676500783341615 2:0.804758241454594 3:-0.0642091078911513 4:0.1402705435750966 5:-0.9215322030628859 6:0.7951173116514345 7:-0.994819896842561 8:0.2382406912119326 9:0.6634166177958731 10:0.7623222578718651 +5.017247792012723 1:-0.5925393497160352 2:0.48506599831456443 3:-0.5079795649118319 4:0.6668553329827696 5:-0.1103174867779837 6:0.7048535526809607 7:-0.9819230894106692 8:0.19609620625274982 9:0.5173985272313828 10:-0.11269849619148875 +6.201510810634532 1:-0.6802942101330738 2:0.898957584078176 3:0.853293387559251 4:0.6089336185656065 5:-0.9352626288322801 6:0.3208583332890447 7:-0.964481544931127 8:-0.8294773786068643 9:-0.8817311989413614 10:0.5165364663580934 +19.174935630244647 1:-0.20026105252200788 2:0.7276178994821614 3:0.7748716685190951 4:-0.7423420145576229 5:0.13147770471985032 6:-0.8382015712894606 7:0.021760992104270294 8:-0.24586987823702944 9:-0.05958177281299326 10:0.47347236224860834 +-14.822152909751189 1:0.7255660700197897 2:-0.22751988933383926 3:-0.08409197084114317 4:0.072750455428638 5:0.1841692073989072 6:-0.33838406658716513 7:-0.44701963574290526 8:0.5031210959133143 9:0.09640858549693743 10:0.9857351194637847 +-6.310082095945472 1:-0.7692076133438608 2:0.8533601511731044 3:0.676268298275629 4:-0.783895030001512 5:-0.8195462819549715 6:0.3963101354895673 7:-0.6254922461977397 8:-0.7521135990258581 9:-0.8032003997516024 10:0.8388672800826487 +8.853802632714807 1:0.46950948246522195 2:-0.6148693581037883 3:0.028739220735170656 4:-0.024281643566285815 5:-0.3495458137792231 6:-0.12347196435522867 7:0.5253894065203333 8:0.5100713458262918 9:0.63975795701667 10:0.08644353314625053 +-10.293714040655924 1:-0.17971950768550893 2:-0.6621720204354751 3:0.888036885802737 4:-0.04977483590350751 5:-0.8964991391283221 6:0.6873490822438724 7:0.42369087852118836 8:0.48972554317650663 9:0.8617233178519317 10:-0.8348331836605276 +0.23985611568891863 1:0.050526696983213215 2:0.8544297176525815 3:0.8586358519997579 4:-0.021299752441110487 5:0.2606696929560939 6:-0.39446486150105997 7:-0.4166234435381613 8:-0.6097643266459343 9:0.46633996256010146 10:-0.22521646199731027 +21.57719950299147 1:-0.5878491135126271 2:0.802134056970349 3:-0.5471017580843434 4:0.6067966843473331 5:-0.691712219323007 6:0.7814323754276735 7:0.31689445927290016 8:-0.1668780061940922 9:0.5285692389527452 10:0.8027091025203246 +-0.7836538830323514 1:0.5766794801558166 2:0.8281463568384935 3:0.5087453132796032 4:0.5212853344036532 5:0.6294700781054074 6:-0.9385097739886943 7:-0.13127371407538302 8:0.9845390503404141 9:-0.7224166213906742 10:-0.11155327354295896 +6.710413649604831 1:-0.6919803228062729 2:-0.6526904017578161 3:-0.34211291948607014 4:0.9094842803341618 5:-0.9454398661995895 6:0.3780766512494227 7:0.5823385348738088 8:0.8817830051841733 9:-0.514843382774189 10:0.32579701113259296 +5.384747201245483 1:-0.9661857672086316 2:-0.519769534339731 3:-0.4466396856529564 4:-0.4370113024678448 5:-0.6397400687811474 6:0.08225309277403725 7:-0.25936524603970756 8:-0.1711463274766858 9:-0.42848099098115755 10:-0.8096854737357237 +7.688509532916731 1:0.3892872094452817 2:-0.13306620868059982 3:-0.932974891205117 4:-0.8921357494146682 5:0.4806996560679244 6:-0.21500288444218696 7:-0.8911268070046585 8:-0.9510264953215406 9:0.1899740993687098 10:-0.43944320580463536 +2.2546997585565296 1:-0.5963883101717473 2:-0.01115153603404151 3:0.8781871380140298 4:0.7736250964135891 5:-0.7325745711528668 6:0.2518631794989008 7:0.5760249284318746 8:0.8690107952725199 9:0.02320853138646095 10:0.08570951531344417 +5.597710012706039 1:-0.5323512235815979 2:0.03366944321271936 3:0.5143537675853551 4:0.28471250955283445 5:0.4012202634439719 6:0.12032039285431151 7:-0.08108716844967812 8:0.30231384371011294 9:0.03259115565303028 10:0.9567467516929173 +-12.977848725392104 1:-0.5908891529017144 2:-0.7678208242918028 3:0.8512434510178621 4:-0.14910196410347298 5:0.6250260229199651 6:0.5393378705290228 7:-0.9573580597625002 8:-0.864881502860934 9:0.4175735160503429 10:0.4872169215922426 +10.35887243981476 1:-0.09126023790482862 2:0.18852634121926526 3:-0.13523918100503107 4:0.8333842692409983 5:-0.6015442103644761 6:0.5347736461652235 7:-0.823489760471118 8:0.5562688292037381 9:-0.807478561291906 10:-0.666881464988351 +0.4250502150408626 1:0.7771717566171905 2:-0.8729202752916785 3:-0.25782888805127024 4:-0.13605474993771205 5:0.5911781118120025 6:-0.8444023967853633 7:0.6787302541469229 8:-0.5444299313083194 9:0.356121883138657 10:-0.8845333845080687 +-0.8743487925900991 1:-0.9087681208947878 2:-0.292625136739453 3:-0.35113758823291774 4:-0.705933223571676 5:-0.6882289471031144 6:0.8350131255297044 7:-0.7659016065609232 8:0.11400114955653207 9:-0.9466143658505732 10:-0.5033643125229932 +-5.615143641864686 1:-0.6688289820084299 2:-0.4623159855015393 3:0.012827807007503855 4:-0.44521264878006117 5:-0.5563111031201406 6:-0.6065295981983794 7:0.3806712426786838 8:-0.11317152118817408 9:0.507896127467435 10:-0.8487801189674464 +-0.1829397047693725 1:0.09377558075225512 2:0.5774384503027374 3:-0.7104684187448009 4:-0.07285914169135976 5:-0.8797920488335114 6:0.6099615504974201 7:-0.8047440624324915 8:-0.6877856114263066 9:0.5843004021777447 10:0.5190581455348131 +18.479680552020344 1:0.9635517137863321 2:0.9954507816218203 3:0.11959899129360774 4:0.3753283274192787 5:-0.9386713095183621 6:0.0926833703812433 7:0.48003949462701323 8:0.9432769781973132 9:-0.9637036991931129 10:-0.4064407447273508 +1.3850645873427236 1:0.14476184437006356 2:-0.11280617018445871 3:-0.4385084538142101 4:-0.5961619435136434 5:0.419554626795412 6:-0.5047767472761191 7:0.457180284958592 8:-0.9129360314541999 9:-0.6320022059786656 10:-0.44989608519659363 diff --git a/data/mllib/sample_movielens_data.txt b/data/mllib/sample_movielens_data.txt new file mode 100644 index 0000000000000..f0eee19875f76 --- /dev/null +++ b/data/mllib/sample_movielens_data.txt @@ -0,0 +1,1501 @@ +0::2::3 +0::3::1 +0::5::2 +0::9::4 +0::11::1 +0::12::2 +0::15::1 +0::17::1 +0::19::1 +0::21::1 +0::23::1 +0::26::3 +0::27::1 +0::28::1 +0::29::1 +0::30::1 +0::31::1 +0::34::1 +0::37::1 +0::41::2 +0::44::1 +0::45::2 +0::46::1 +0::47::1 +0::48::1 +0::50::1 +0::51::1 +0::54::1 +0::55::1 +0::59::2 +0::61::2 +0::64::1 +0::67::1 +0::68::1 +0::69::1 +0::71::1 +0::72::1 +0::77::2 +0::79::1 +0::83::1 +0::87::1 +0::89::2 +0::91::3 +0::92::4 +0::94::1 +0::95::2 +0::96::1 +0::98::1 +0::99::1 +1::2::2 +1::3::1 +1::4::2 +1::6::1 +1::9::3 +1::12::1 +1::13::1 +1::14::1 +1::16::1 +1::19::1 +1::21::3 +1::27::1 +1::28::3 +1::33::1 +1::36::2 +1::37::1 +1::40::1 +1::41::2 +1::43::1 +1::44::1 +1::47::1 +1::50::1 +1::54::1 +1::56::2 +1::57::1 +1::58::1 +1::60::1 +1::62::4 +1::63::1 +1::67::1 +1::68::4 +1::70::2 +1::72::1 +1::73::1 +1::74::2 +1::76::1 +1::77::3 +1::78::1 +1::81::1 +1::82::1 +1::85::3 +1::86::2 +1::88::2 +1::91::1 +1::92::2 +1::93::1 +1::94::2 +1::96::1 +1::97::1 +2::4::3 +2::6::1 +2::8::5 +2::9::1 +2::10::1 +2::12::3 +2::13::1 +2::15::2 +2::18::2 +2::19::4 +2::22::1 +2::26::1 +2::28::1 +2::34::4 +2::35::1 +2::37::5 +2::38::1 +2::39::5 +2::40::4 +2::47::1 +2::50::1 +2::52::2 +2::54::1 +2::55::1 +2::57::2 +2::58::2 +2::59::1 +2::61::1 +2::62::1 +2::64::1 +2::65::1 +2::66::3 +2::68::1 +2::71::3 +2::76::1 +2::77::1 +2::78::1 +2::80::1 +2::83::5 +2::85::1 +2::87::2 +2::88::1 +2::89::4 +2::90::1 +2::92::4 +2::93::5 +3::0::1 +3::1::1 +3::2::1 +3::7::3 +3::8::3 +3::9::1 +3::14::1 +3::15::1 +3::16::1 +3::18::4 +3::19::1 +3::24::3 +3::26::1 +3::29::3 +3::33::1 +3::34::3 +3::35::1 +3::36::3 +3::37::1 +3::38::2 +3::43::1 +3::44::1 +3::46::1 +3::47::1 +3::51::5 +3::52::3 +3::56::1 +3::58::1 +3::60::3 +3::62::1 +3::65::2 +3::66::1 +3::67::1 +3::68::2 +3::70::1 +3::72::2 +3::76::3 +3::79::3 +3::80::4 +3::81::1 +3::83::1 +3::84::1 +3::86::1 +3::87::2 +3::88::4 +3::89::1 +3::91::1 +3::94::3 +4::1::1 +4::6::1 +4::8::1 +4::9::1 +4::10::1 +4::11::1 +4::12::1 +4::13::1 +4::14::2 +4::15::1 +4::17::1 +4::20::1 +4::22::1 +4::23::1 +4::24::1 +4::29::4 +4::30::1 +4::31::1 +4::34::1 +4::35::1 +4::36::1 +4::39::2 +4::40::3 +4::41::4 +4::43::2 +4::44::1 +4::45::1 +4::46::1 +4::47::1 +4::49::2 +4::50::1 +4::51::1 +4::52::4 +4::54::1 +4::55::1 +4::60::3 +4::61::1 +4::62::4 +4::63::3 +4::65::1 +4::67::2 +4::69::1 +4::70::4 +4::71::1 +4::73::1 +4::78::1 +4::84::1 +4::85::1 +4::87::3 +4::88::3 +4::89::2 +4::96::1 +4::97::1 +4::98::1 +4::99::1 +5::0::1 +5::1::1 +5::4::1 +5::5::1 +5::8::1 +5::9::3 +5::10::2 +5::13::3 +5::15::1 +5::19::1 +5::20::3 +5::21::2 +5::23::3 +5::27::1 +5::28::1 +5::29::1 +5::31::1 +5::36::3 +5::38::2 +5::39::1 +5::42::1 +5::48::3 +5::49::4 +5::50::3 +5::51::1 +5::52::1 +5::54::1 +5::55::5 +5::56::3 +5::58::1 +5::60::1 +5::61::1 +5::64::3 +5::65::2 +5::68::4 +5::70::1 +5::71::1 +5::72::1 +5::74::1 +5::79::1 +5::81::2 +5::84::1 +5::85::1 +5::86::1 +5::88::1 +5::90::4 +5::91::2 +5::95::2 +5::99::1 +6::0::1 +6::1::1 +6::2::3 +6::5::1 +6::6::1 +6::9::1 +6::10::1 +6::15::2 +6::16::2 +6::17::1 +6::18::1 +6::20::1 +6::21::1 +6::22::1 +6::24::1 +6::25::5 +6::26::1 +6::28::1 +6::30::1 +6::33::1 +6::38::1 +6::39::1 +6::43::4 +6::44::1 +6::45::1 +6::48::1 +6::49::1 +6::50::1 +6::53::1 +6::54::1 +6::55::1 +6::56::1 +6::58::4 +6::59::1 +6::60::1 +6::61::3 +6::63::3 +6::66::1 +6::67::3 +6::68::1 +6::69::1 +6::71::2 +6::73::1 +6::75::1 +6::77::1 +6::79::1 +6::81::1 +6::84::1 +6::85::3 +6::86::1 +6::87::1 +6::88::1 +6::89::1 +6::91::2 +6::94::1 +6::95::2 +6::96::1 +7::1::1 +7::2::2 +7::3::1 +7::4::1 +7::7::1 +7::10::1 +7::11::2 +7::14::2 +7::15::1 +7::16::1 +7::18::1 +7::21::1 +7::22::1 +7::23::1 +7::25::5 +7::26::1 +7::29::4 +7::30::1 +7::31::3 +7::32::1 +7::33::1 +7::35::1 +7::37::2 +7::39::3 +7::40::2 +7::42::2 +7::44::1 +7::45::2 +7::47::4 +7::48::1 +7::49::1 +7::53::1 +7::54::1 +7::55::1 +7::56::1 +7::59::1 +7::61::2 +7::62::3 +7::63::2 +7::66::1 +7::67::3 +7::74::1 +7::75::1 +7::76::3 +7::77::1 +7::81::1 +7::82::1 +7::84::2 +7::85::4 +7::86::1 +7::92::2 +7::96::1 +7::97::1 +7::98::1 +8::0::1 +8::2::4 +8::3::2 +8::4::2 +8::5::1 +8::7::1 +8::9::1 +8::11::1 +8::15::1 +8::18::1 +8::19::1 +8::21::1 +8::29::5 +8::31::3 +8::33::1 +8::35::1 +8::36::1 +8::40::2 +8::44::1 +8::45::1 +8::50::1 +8::51::1 +8::52::5 +8::53::5 +8::54::1 +8::55::1 +8::56::1 +8::58::4 +8::60::3 +8::62::4 +8::64::1 +8::67::3 +8::69::1 +8::71::1 +8::72::3 +8::77::3 +8::78::1 +8::79::1 +8::83::1 +8::85::5 +8::86::1 +8::88::1 +8::90::1 +8::92::2 +8::95::4 +8::96::3 +8::97::1 +8::98::1 +8::99::1 +9::2::3 +9::3::1 +9::4::1 +9::5::1 +9::6::1 +9::7::5 +9::9::1 +9::12::1 +9::14::3 +9::15::1 +9::19::1 +9::21::1 +9::22::1 +9::24::1 +9::25::1 +9::26::1 +9::30::3 +9::32::4 +9::35::2 +9::36::2 +9::37::2 +9::38::1 +9::39::1 +9::43::3 +9::49::5 +9::50::3 +9::53::1 +9::54::1 +9::58::1 +9::59::1 +9::60::1 +9::61::1 +9::63::3 +9::64::3 +9::68::1 +9::69::1 +9::70::3 +9::71::1 +9::73::2 +9::75::1 +9::77::2 +9::81::2 +9::82::1 +9::83::1 +9::84::1 +9::86::1 +9::87::4 +9::88::1 +9::90::3 +9::94::2 +9::95::3 +9::97::2 +9::98::1 +10::0::3 +10::2::4 +10::4::3 +10::7::1 +10::8::1 +10::10::1 +10::13::2 +10::14::1 +10::16::2 +10::17::1 +10::18::1 +10::21::1 +10::22::1 +10::24::1 +10::25::3 +10::28::1 +10::35::1 +10::36::1 +10::37::1 +10::38::1 +10::39::1 +10::40::4 +10::41::2 +10::42::3 +10::43::1 +10::49::3 +10::50::1 +10::51::1 +10::52::1 +10::55::2 +10::56::1 +10::58::1 +10::63::1 +10::66::1 +10::67::2 +10::68::1 +10::75::1 +10::77::1 +10::79::1 +10::86::1 +10::89::3 +10::90::1 +10::97::1 +10::98::1 +11::0::1 +11::6::2 +11::9::1 +11::10::1 +11::11::1 +11::12::1 +11::13::4 +11::16::1 +11::18::5 +11::19::4 +11::20::1 +11::21::1 +11::22::1 +11::23::5 +11::25::1 +11::27::5 +11::30::5 +11::32::5 +11::35::3 +11::36::2 +11::37::2 +11::38::4 +11::39::1 +11::40::1 +11::41::1 +11::43::2 +11::45::1 +11::47::1 +11::48::5 +11::50::4 +11::51::3 +11::59::1 +11::61::1 +11::62::1 +11::64::1 +11::66::4 +11::67::1 +11::69::5 +11::70::1 +11::71::3 +11::72::3 +11::75::3 +11::76::1 +11::77::1 +11::78::1 +11::79::5 +11::80::3 +11::81::4 +11::82::1 +11::86::1 +11::88::1 +11::89::1 +11::90::4 +11::94::2 +11::97::3 +11::99::1 +12::2::1 +12::4::1 +12::6::1 +12::7::3 +12::8::1 +12::14::1 +12::15::2 +12::16::4 +12::17::5 +12::18::2 +12::21::1 +12::22::2 +12::23::3 +12::24::1 +12::25::1 +12::27::5 +12::30::2 +12::31::4 +12::35::5 +12::38::1 +12::41::1 +12::44::2 +12::45::1 +12::50::4 +12::51::1 +12::52::1 +12::53::1 +12::54::1 +12::56::2 +12::57::1 +12::60::1 +12::63::1 +12::64::5 +12::66::3 +12::67::1 +12::70::1 +12::72::1 +12::74::1 +12::75::1 +12::77::1 +12::78::1 +12::79::3 +12::82::2 +12::83::1 +12::84::1 +12::85::1 +12::86::1 +12::87::1 +12::88::1 +12::91::3 +12::92::1 +12::94::4 +12::95::2 +12::96::1 +12::98::2 +13::0::1 +13::3::1 +13::4::2 +13::5::1 +13::6::1 +13::12::1 +13::14::2 +13::15::1 +13::17::1 +13::18::3 +13::20::1 +13::21::1 +13::22::1 +13::26::1 +13::27::1 +13::29::3 +13::31::1 +13::33::1 +13::40::2 +13::43::2 +13::44::1 +13::45::1 +13::49::1 +13::51::1 +13::52::2 +13::53::3 +13::54::1 +13::62::1 +13::63::2 +13::64::1 +13::68::1 +13::71::1 +13::72::3 +13::73::1 +13::74::3 +13::77::2 +13::78::1 +13::79::2 +13::83::3 +13::85::1 +13::86::1 +13::87::2 +13::88::2 +13::90::1 +13::93::4 +13::94::1 +13::98::1 +13::99::1 +14::1::1 +14::3::3 +14::4::1 +14::5::1 +14::6::1 +14::7::1 +14::9::1 +14::10::1 +14::11::1 +14::12::1 +14::13::1 +14::14::3 +14::15::1 +14::16::1 +14::17::1 +14::20::1 +14::21::1 +14::24::1 +14::25::2 +14::27::1 +14::28::1 +14::29::5 +14::31::3 +14::34::1 +14::36::1 +14::37::2 +14::39::2 +14::40::1 +14::44::1 +14::45::1 +14::47::3 +14::48::1 +14::49::1 +14::51::1 +14::52::5 +14::53::3 +14::54::1 +14::55::1 +14::56::1 +14::62::4 +14::63::5 +14::67::3 +14::68::1 +14::69::3 +14::71::1 +14::72::4 +14::73::1 +14::76::5 +14::79::1 +14::82::1 +14::83::1 +14::88::1 +14::93::3 +14::94::1 +14::95::2 +14::96::4 +14::98::1 +15::0::1 +15::1::4 +15::2::1 +15::5::2 +15::6::1 +15::7::1 +15::13::1 +15::14::1 +15::15::1 +15::17::2 +15::19::2 +15::22::2 +15::23::2 +15::25::1 +15::26::3 +15::27::1 +15::28::2 +15::29::1 +15::32::1 +15::33::2 +15::34::1 +15::35::2 +15::36::1 +15::37::1 +15::39::1 +15::42::1 +15::46::5 +15::48::2 +15::50::2 +15::51::1 +15::52::1 +15::58::1 +15::62::1 +15::64::3 +15::65::2 +15::72::1 +15::73::1 +15::74::1 +15::79::1 +15::80::1 +15::81::1 +15::82::2 +15::85::1 +15::87::1 +15::91::2 +15::96::1 +15::97::1 +15::98::3 +16::2::1 +16::5::3 +16::6::2 +16::7::1 +16::9::1 +16::12::1 +16::14::1 +16::15::1 +16::19::1 +16::21::2 +16::29::4 +16::30::2 +16::32::1 +16::34::1 +16::36::1 +16::38::1 +16::46::1 +16::47::3 +16::48::1 +16::49::1 +16::50::1 +16::51::5 +16::54::5 +16::55::1 +16::56::2 +16::57::1 +16::60::1 +16::63::2 +16::65::1 +16::67::1 +16::72::1 +16::74::1 +16::80::1 +16::81::1 +16::82::1 +16::85::5 +16::86::1 +16::90::5 +16::91::1 +16::93::1 +16::94::3 +16::95::2 +16::96::3 +16::98::3 +16::99::1 +17::2::1 +17::3::1 +17::6::1 +17::10::4 +17::11::1 +17::13::2 +17::17::5 +17::19::1 +17::20::5 +17::22::4 +17::28::1 +17::29::1 +17::33::1 +17::34::1 +17::35::2 +17::37::1 +17::38::1 +17::45::1 +17::46::5 +17::47::1 +17::49::3 +17::51::1 +17::55::5 +17::56::3 +17::57::1 +17::58::1 +17::59::1 +17::60::1 +17::63::1 +17::66::1 +17::68::4 +17::69::1 +17::70::1 +17::72::1 +17::73::3 +17::78::1 +17::79::1 +17::82::2 +17::84::1 +17::90::5 +17::91::3 +17::92::1 +17::93::1 +17::94::4 +17::95::2 +17::97::1 +18::1::1 +18::4::3 +18::5::2 +18::6::1 +18::7::1 +18::10::1 +18::11::4 +18::12::2 +18::13::1 +18::15::1 +18::18::1 +18::20::1 +18::21::2 +18::22::1 +18::23::2 +18::25::1 +18::26::1 +18::27::1 +18::28::5 +18::29::1 +18::31::1 +18::32::1 +18::36::1 +18::38::5 +18::39::5 +18::40::1 +18::42::1 +18::43::1 +18::44::4 +18::46::1 +18::47::1 +18::48::1 +18::51::2 +18::55::1 +18::56::1 +18::57::1 +18::62::1 +18::63::1 +18::66::3 +18::67::1 +18::70::1 +18::75::1 +18::76::3 +18::77::1 +18::80::3 +18::81::3 +18::82::1 +18::83::5 +18::84::1 +18::97::1 +18::98::1 +18::99::2 +19::0::1 +19::1::1 +19::2::1 +19::4::1 +19::6::2 +19::11::1 +19::12::1 +19::14::1 +19::23::1 +19::26::1 +19::31::1 +19::32::4 +19::33::1 +19::34::1 +19::37::1 +19::38::1 +19::41::1 +19::43::1 +19::45::1 +19::48::1 +19::49::1 +19::50::2 +19::53::2 +19::54::3 +19::55::1 +19::56::2 +19::58::1 +19::61::1 +19::62::1 +19::63::1 +19::64::1 +19::65::1 +19::69::2 +19::72::1 +19::74::3 +19::76::1 +19::78::1 +19::79::1 +19::81::1 +19::82::1 +19::84::1 +19::86::1 +19::87::2 +19::90::4 +19::93::1 +19::94::4 +19::95::2 +19::96::1 +19::98::4 +20::0::1 +20::1::1 +20::2::2 +20::4::2 +20::6::1 +20::8::1 +20::12::1 +20::21::2 +20::22::5 +20::24::2 +20::25::1 +20::26::1 +20::29::2 +20::30::2 +20::32::2 +20::39::1 +20::40::1 +20::41::2 +20::45::2 +20::48::1 +20::50::1 +20::51::3 +20::53::3 +20::55::1 +20::57::2 +20::60::1 +20::61::1 +20::64::1 +20::66::1 +20::70::2 +20::72::1 +20::73::2 +20::75::4 +20::76::1 +20::77::4 +20::78::1 +20::79::1 +20::84::2 +20::85::2 +20::88::3 +20::89::1 +20::90::3 +20::91::1 +20::92::2 +20::93::1 +20::94::4 +20::97::1 +21::0::1 +21::2::4 +21::3::1 +21::7::2 +21::11::1 +21::12::1 +21::13::1 +21::14::3 +21::17::1 +21::19::1 +21::20::1 +21::21::1 +21::22::1 +21::23::1 +21::24::1 +21::27::1 +21::29::5 +21::30::2 +21::38::1 +21::40::2 +21::43::3 +21::44::1 +21::45::1 +21::46::1 +21::48::1 +21::51::1 +21::53::5 +21::54::1 +21::55::1 +21::56::1 +21::58::3 +21::59::3 +21::64::1 +21::66::1 +21::68::1 +21::71::1 +21::73::1 +21::74::4 +21::80::1 +21::81::1 +21::83::1 +21::84::1 +21::85::3 +21::87::4 +21::89::2 +21::92::2 +21::96::3 +21::99::1 +22::0::1 +22::3::2 +22::5::2 +22::6::2 +22::9::1 +22::10::1 +22::11::1 +22::13::1 +22::14::1 +22::16::1 +22::18::3 +22::19::1 +22::22::5 +22::25::1 +22::26::1 +22::29::3 +22::30::5 +22::32::4 +22::33::1 +22::35::1 +22::36::3 +22::37::1 +22::40::1 +22::41::3 +22::44::1 +22::45::2 +22::48::1 +22::51::5 +22::55::1 +22::56::2 +22::60::3 +22::61::1 +22::62::4 +22::63::1 +22::65::1 +22::66::1 +22::68::4 +22::69::4 +22::70::3 +22::71::1 +22::74::5 +22::75::5 +22::78::1 +22::80::3 +22::81::1 +22::82::1 +22::84::1 +22::86::1 +22::87::3 +22::88::5 +22::90::2 +22::92::3 +22::95::2 +22::96::2 +22::98::4 +22::99::1 +23::0::1 +23::2::1 +23::4::1 +23::6::2 +23::10::4 +23::12::1 +23::13::4 +23::14::1 +23::15::1 +23::18::4 +23::22::2 +23::23::4 +23::24::1 +23::25::1 +23::26::1 +23::27::5 +23::28::1 +23::29::1 +23::30::4 +23::32::5 +23::33::2 +23::36::3 +23::37::1 +23::38::1 +23::39::1 +23::43::1 +23::48::5 +23::49::5 +23::50::4 +23::53::1 +23::55::5 +23::57::1 +23::59::1 +23::60::1 +23::61::1 +23::64::4 +23::65::5 +23::66::2 +23::67::1 +23::68::3 +23::69::1 +23::72::1 +23::73::3 +23::77::1 +23::82::2 +23::83::1 +23::84::1 +23::85::1 +23::87::3 +23::88::1 +23::95::2 +23::97::1 +24::4::1 +24::6::3 +24::7::1 +24::10::2 +24::12::1 +24::15::1 +24::19::1 +24::24::1 +24::27::3 +24::30::5 +24::31::1 +24::32::3 +24::33::1 +24::37::1 +24::39::1 +24::40::1 +24::42::1 +24::43::3 +24::45::2 +24::46::1 +24::47::1 +24::48::1 +24::49::1 +24::50::1 +24::52::5 +24::57::1 +24::59::4 +24::63::4 +24::65::1 +24::66::1 +24::67::1 +24::68::3 +24::69::5 +24::71::1 +24::72::4 +24::77::4 +24::78::1 +24::80::1 +24::82::1 +24::84::1 +24::86::1 +24::87::1 +24::88::2 +24::89::1 +24::90::5 +24::91::1 +24::92::1 +24::94::2 +24::95::1 +24::96::5 +24::98::1 +24::99::1 +25::1::3 +25::2::1 +25::7::1 +25::9::1 +25::12::3 +25::16::3 +25::17::1 +25::18::1 +25::20::1 +25::22::1 +25::23::1 +25::26::2 +25::29::1 +25::30::1 +25::31::2 +25::33::4 +25::34::3 +25::35::2 +25::36::1 +25::37::1 +25::40::1 +25::41::1 +25::43::1 +25::47::4 +25::50::1 +25::51::1 +25::53::1 +25::56::1 +25::58::2 +25::64::2 +25::67::2 +25::68::1 +25::70::1 +25::71::4 +25::73::1 +25::74::1 +25::76::1 +25::79::1 +25::82::1 +25::84::2 +25::85::1 +25::91::3 +25::92::1 +25::94::1 +25::95::1 +25::97::2 +26::0::1 +26::1::1 +26::2::1 +26::3::1 +26::4::4 +26::5::2 +26::6::3 +26::7::5 +26::13::3 +26::14::1 +26::16::1 +26::18::3 +26::20::1 +26::21::3 +26::22::5 +26::23::5 +26::24::5 +26::27::1 +26::31::1 +26::35::1 +26::36::4 +26::40::1 +26::44::1 +26::45::2 +26::47::1 +26::48::1 +26::49::3 +26::50::2 +26::52::1 +26::54::4 +26::55::1 +26::57::3 +26::58::1 +26::61::1 +26::62::2 +26::66::1 +26::68::4 +26::71::1 +26::73::4 +26::76::1 +26::81::3 +26::85::1 +26::86::3 +26::88::5 +26::91::1 +26::94::5 +26::95::1 +26::96::1 +26::97::1 +27::0::1 +27::9::1 +27::10::1 +27::18::4 +27::19::3 +27::20::1 +27::22::2 +27::24::2 +27::25::1 +27::27::3 +27::28::1 +27::29::1 +27::31::1 +27::33::3 +27::40::1 +27::42::1 +27::43::1 +27::44::3 +27::45::1 +27::51::3 +27::52::1 +27::55::3 +27::57::1 +27::59::1 +27::60::1 +27::61::1 +27::64::1 +27::66::3 +27::68::1 +27::70::1 +27::71::2 +27::72::1 +27::75::3 +27::78::1 +27::80::3 +27::82::1 +27::83::3 +27::86::1 +27::87::2 +27::90::1 +27::91::1 +27::92::1 +27::93::1 +27::94::2 +27::95::1 +27::98::1 +28::0::3 +28::1::1 +28::2::4 +28::3::1 +28::6::1 +28::7::1 +28::12::5 +28::13::2 +28::14::1 +28::15::1 +28::17::1 +28::19::3 +28::20::1 +28::23::3 +28::24::3 +28::27::1 +28::29::1 +28::33::1 +28::34::1 +28::36::1 +28::38::2 +28::39::2 +28::44::1 +28::45::1 +28::49::4 +28::50::1 +28::52::1 +28::54::1 +28::56::1 +28::57::3 +28::58::1 +28::59::1 +28::60::1 +28::62::3 +28::63::1 +28::65::1 +28::75::1 +28::78::1 +28::81::5 +28::82::4 +28::83::1 +28::85::1 +28::88::2 +28::89::4 +28::90::1 +28::92::5 +28::94::1 +28::95::2 +28::98::1 +28::99::1 +29::3::1 +29::4::1 +29::5::1 +29::7::2 +29::9::1 +29::10::3 +29::11::1 +29::13::3 +29::14::1 +29::15::1 +29::17::3 +29::19::3 +29::22::3 +29::23::4 +29::25::1 +29::29::1 +29::31::1 +29::32::4 +29::33::2 +29::36::2 +29::38::3 +29::39::1 +29::42::1 +29::46::5 +29::49::3 +29::51::2 +29::59::1 +29::61::1 +29::62::1 +29::67::1 +29::68::3 +29::69::1 +29::70::1 +29::74::1 +29::75::1 +29::79::2 +29::80::1 +29::81::2 +29::83::1 +29::85::1 +29::86::1 +29::90::4 +29::93::1 +29::94::4 +29::97::1 +29::99::1 diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala index 4001908c98015..56b02b65d8724 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala @@ -29,8 +29,9 @@ import org.apache.spark.mllib.optimization.{SquaredL2Updater, L1Updater} /** * An example app for binary classification. Run with * {{{ - * ./bin/run-example org.apache.spark.examples.mllib.BinaryClassification + * bin/run-example org.apache.spark.examples.mllib.BinaryClassification * }}} + * A synthetic dataset is located at `data/mllib/sample_binary_classification_data.txt`. * If you use it as a template to create your own app, please use `spark-submit` to submit your app. */ object BinaryClassification { @@ -81,6 +82,15 @@ object BinaryClassification { .required() .text("input paths to labeled examples in LIBSVM format") .action((x, c) => c.copy(input = x)) + note( + """ + |For example, the following command runs this app on a synthetic dataset: + | + | bin/spark-submit --class org.apache.spark.examples.mllib.BinaryClassification \ + | examples/target/scala-*/spark-examples-*.jar \ + | --algorithm LR --regType L2 --regParam 1.0 \ + | data/mllib/sample_binary_classification_data.txt + """.stripMargin) } parser.parse(args, defaultParams).map { params => diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala index 658d370f8656e..4811bb70e4b28 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala @@ -28,8 +28,9 @@ import org.apache.spark.mllib.optimization.{SimpleUpdater, SquaredL2Updater, L1U /** * An example app for linear regression. Run with * {{{ - * ./bin/run-example org.apache.spark.examples.mllib.LinearRegression + * bin/run-example org.apache.spark.examples.mllib.LinearRegression * }}} + * A synthetic dataset can be found at `data/mllib/sample_linear_regression_data.txt`. * If you use it as a template to create your own app, please use `spark-submit` to submit your app. */ object LinearRegression extends App { @@ -68,6 +69,14 @@ object LinearRegression extends App { .required() .text("input paths to labeled examples in LIBSVM format") .action((x, c) => c.copy(input = x)) + note( + """ + |For example, the following command runs this app on a synthetic dataset: + | + | bin/spark-submit --class org.apache.spark.examples.mllib.LinearRegression \ + | examples/target/scala-*/spark-examples-*.jar \ + | data/mllib/sample_linear_regression_data.txt + """.stripMargin) } parser.parse(args, defaultParams).map { params => diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala index 0e4447e0de24f..6eb41e7ba36fb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala @@ -29,6 +29,12 @@ import org.apache.spark.serializer.{KryoSerializer, KryoRegistrator} /** * An example app for ALS on MovieLens data (http://grouplens.org/datasets/movielens/). + * Run with + * {{{ + * bin/run-example org.apache.spark.examples.mllib.MovieLensALS + * }}} + * A synthetic dataset in MovieLens format can be found at `data/mllib/sample_movielens_data.txt`. + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. */ object MovieLensALS { @@ -70,6 +76,15 @@ object MovieLensALS { .required() .text("input paths to a MovieLens dataset of ratings") .action((x, c) => c.copy(input = x)) + note( + """ + |For example, the following command runs this app on a synthetic dataset: + | + | bin/spark-submit --class org.apache.spark.examples.mllib.MovieLensALS \ + | examples/target/scala-*/spark-examples-*.jar \ + | --rank 5 --numIterations 20 --lambda 1.0 --kryo \ + | data/mllib/sample_movielens_data.txt + """.stripMargin) } parser.parse(args, defaultParams).map { params => From 52eb54d02403a3c37d84b9da7cc1cdb261048cf8 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 19 May 2014 22:36:24 -0700 Subject: [PATCH 398/641] [Spark 1877] ClassNotFoundException when loading RDD with serialized objects Updated version of #821 Author: Tathagata Das Author: Ghidireac Closes #835 from tdas/SPARK-1877 and squashes the following commits: f346f71 [Tathagata Das] Addressed Patrick's comments. fee0c5d [Ghidireac] SPARK-1877: ClassNotFoundException when loading RDD with serialized objects --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 634c10c33fc86..49737fa4be56b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -718,7 +718,7 @@ class SparkContext(config: SparkConf) extends Logging { minPartitions: Int = defaultMinPartitions ): RDD[T] = { sequenceFile(path, classOf[NullWritable], classOf[BytesWritable], minPartitions) - .flatMap(x => Utils.deserialize[Array[T]](x._2.getBytes)) + .flatMap(x => Utils.deserialize[Array[T]](x._2.getBytes, Utils.getContextOrSparkClassLoader)) } protected[spark] def checkpointFile[T: ClassTag]( From 7f0cfe47f4709843d70ceccc25dee7551206ce0d Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 20 May 2014 10:27:12 -0700 Subject: [PATCH 399/641] [Hotfix] Blacklisted flaky HiveCompatibility test `lateral_view_outer` query sometimes returns a different set of 10 rows. Author: Tathagata Das Closes #838 from tdas/hive-test-fix2 and squashes the following commits: 9128a0d [Tathagata Das] Blacklisted flaky HiveCompatibility test. --- .../spark/sql/hive/execution/HiveCompatibilitySuite.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index d83732b51e9c2..9031abf733cd4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -169,7 +169,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "archive_corrupt", // No support for case sensitivity is resolution using hive properties atm. - "case_sensitivity" + "case_sensitivity", + + // Flaky test, Hive sometimes returns different set of 10 rows. + "lateral_view_outer" ) /** @@ -476,7 +479,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "join_view", "lateral_view", "lateral_view_cp", - "lateral_view_outer", "lateral_view_ppd", "lineage1", "literal_double", From 6e337380fc47071fc7fb28d744e8209c729fe1e9 Mon Sep 17 00:00:00 2001 From: Sumedh Mungee Date: Wed, 21 May 2014 01:22:25 -0700 Subject: [PATCH 400/641] [SPARK-1250] Fixed misleading comments in bin/pyspark, bin/spark-class Fixed a couple of misleading comments in bin/pyspark and bin/spark-class. The comments make it seem like the script is looking for the Scala installation when in fact it is looking for Spark. Author: Sumedh Mungee Closes #843 from smungee/spark-1250-fix-comments and squashes the following commits: 26870f3 [Sumedh Mungee] [SPARK-1250] Fixed misleading comments in bin/pyspark and bin/spark-class --- bin/pyspark | 2 +- bin/spark-class | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/bin/pyspark b/bin/pyspark index 3908ffe79939b..d0fa56f31913f 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -17,7 +17,7 @@ # limitations under the License. # -# Figure out where the Scala framework is installed +# Figure out where Spark is installed FWDIR="$(cd `dirname $0`/..; pwd)" # Export this as SPARK_HOME diff --git a/bin/spark-class b/bin/spark-class index 2e57295fd0234..e884511010c6c 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -24,7 +24,7 @@ esac SCALA_VERSION=2.10 -# Figure out where the Scala framework is installed +# Figure out where Spark is installed FWDIR="$(cd `dirname $0`/..; pwd)" # Export this as SPARK_HOME From 1014668f2727863fe46f9c75201ee459d093bf0c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 21 May 2014 01:23:34 -0700 Subject: [PATCH 401/641] [Docs] Correct example of creating a new SparkConf The example code on the configuration page currently does not compile. Author: Andrew Or Closes #842 from andrewor14/conf-docs and squashes the following commits: aabff57 [Andrew Or] Correct example of creating a new SparkConf --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index f89040d714773..462a9d9013d98 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -23,7 +23,7 @@ cluster (e.g. master URL and application name), as well as arbitrary key-value p `set()` method. For example, we could initialize an application as follows: {% highlight scala %} -val conf = new SparkConf +val conf = new SparkConf() .setMaster("local") .setAppName("CountingSheep") .set("spark.executor.memory", "1g") From 7c79ef7d43de258ad9a5de15c590132bd78ce8dd Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 21 May 2014 01:25:10 -0700 Subject: [PATCH 402/641] [Minor] Move JdbcRDDSuite to the correct package It was in the wrong package Author: Andrew Or Closes #839 from andrewor14/jdbc-suite and squashes the following commits: f948c5a [Andrew Or] cache -> cache() b215279 [Andrew Or] Move JdbcRDDSuite to the correct package --- .../scala/org/apache/spark/rdd/JdbcRDDSuite.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala index 7c7f69b261a0a..76e317d754ba3 100644 --- a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.rdd import java.sql._ import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.rdd.JdbcRDD +import org.apache.spark.{LocalSparkContext, SparkContext} class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { @@ -35,18 +35,18 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { ID INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1), DATA INTEGER )""") - create.close + create.close() val insert = conn.prepareStatement("INSERT INTO FOO(DATA) VALUES(?)") (1 to 100).foreach { i => insert.setInt(1, i * 2) insert.executeUpdate } - insert.close + insert.close() } catch { case e: SQLException if e.getSQLState == "X0Y32" => // table exists } finally { - conn.close + conn.close() } } @@ -57,7 +57,7 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { () => { DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb") }, "SELECT DATA FROM FOO WHERE ? <= ID AND ID <= ?", 1, 100, 3, - (r: ResultSet) => { r.getInt(1) } ).cache + (r: ResultSet) => { r.getInt(1) } ).cache() assert(rdd.count === 100) assert(rdd.reduce(_+_) === 10100) From ba5d4a99425a2083fea2a9759050c5e770197e23 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 21 May 2014 11:59:05 -0700 Subject: [PATCH 403/641] [Typo] Stoped -> Stopped Author: Andrew Or Closes #847 from andrewor14/yarn-typo and squashes the following commits: c1906af [Andrew Or] Stoped -> Stopped --- .../spark/scheduler/cluster/YarnClientSchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 2924189077b7d..0ac162538fc4b 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 @@ -113,7 +113,7 @@ private[spark] class YarnClientSchedulerBackend( override def stop() { super.stop() client.stop() - logInfo("Stoped") + logInfo("Stopped") } } From f18fd05b513b136363c94adb3e5b841f8bf48134 Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Wed, 21 May 2014 13:26:53 -0700 Subject: [PATCH 404/641] [SPARK-1519] Support minPartitions param of wholeTextFiles() in PySpark Author: Kan Zhang Closes #697 from kanzhang/SPARK-1519 and squashes the following commits: 4f8d1ed [Kan Zhang] [SPARK-1519] Support minPartitions param of wholeTextFiles() in PySpark --- python/pyspark/context.py | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index cac133d0fcf6c..c9ff82d23b3cf 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -211,6 +211,13 @@ def defaultParallelism(self): """ return self._jsc.sc().defaultParallelism() + @property + def defaultMinPartitions(self): + """ + Default min number of partitions for Hadoop RDDs when not given by user + """ + return self._jsc.sc().defaultMinPartitions() + def __del__(self): self.stop() @@ -264,7 +271,7 @@ def textFile(self, name, minPartitions=None): return RDD(self._jsc.textFile(name, minPartitions), self, UTF8Deserializer()) - def wholeTextFiles(self, path): + def wholeTextFiles(self, path, minPartitions=None): """ Read a directory of text files from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system @@ -300,7 +307,8 @@ def wholeTextFiles(self, path): >>> sorted(textFiles.collect()) [(u'.../1.txt', u'1'), (u'.../2.txt', u'2')] """ - return RDD(self._jsc.wholeTextFiles(path), self, + minPartitions = minPartitions or self.defaultMinPartitions + return RDD(self._jsc.wholeTextFiles(path, minPartitions), self, PairDeserializer(UTF8Deserializer(), UTF8Deserializer())) def _checkpointFile(self, name, input_deserializer): From bb88875ad52e8209c25e8350af1fe4b7159086ae Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Wed, 21 May 2014 15:37:47 -0700 Subject: [PATCH 405/641] [SPARK-1889] [SQL] Apply splitConjunctivePredicates to join condition while finding join ke... ...ys. When tables are equi-joined by multiple-keys `HashJoin` should be used, but `CartesianProduct` and then `Filter` are used. The join keys are paired by `And` expression so we need to apply `splitConjunctivePredicates` to join condition while finding join keys. Author: Takuya UESHIN Closes #836 from ueshin/issues/SPARK-1889 and squashes the following commits: fe1c387 [Takuya UESHIN] Apply splitConjunctivePredicates to join condition while finding join keys. --- .../sql/catalyst/planning/patterns.scala | 11 ++++++----- .../spark/sql/execution/PlannerSuite.scala | 19 ++++++++++++++++++- 2 files changed, 24 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 0e3a8a6bd30a8..4544b32958c7e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -129,11 +129,12 @@ object HashFilteredJoin extends Logging with PredicateHelper { // as join keys. def splitPredicates(allPredicates: Seq[Expression], join: Join): Option[ReturnType] = { val Join(left, right, joinType, _) = join - val (joinPredicates, otherPredicates) = allPredicates.partition { - case Equals(l, r) if (canEvaluate(l, left) && canEvaluate(r, right)) || - (canEvaluate(l, right) && canEvaluate(r, left)) => true - case _ => false - } + val (joinPredicates, otherPredicates) = + allPredicates.flatMap(splitConjunctivePredicates).partition { + case Equals(l, r) if (canEvaluate(l, left) && canEvaluate(r, right)) || + (canEvaluate(l, right) && canEvaluate(r, left)) => true + case _ => false + } val joinKeys = joinPredicates.map { case Equals(l, r) if canEvaluate(l, left) && canEvaluate(r, right) => (l, r) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index e24c74a7a5572..c563d63627544 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -21,7 +21,7 @@ import org.scalatest.FunSuite import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.execution import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.planner._ @@ -57,4 +57,21 @@ class PlannerSuite extends FunSuite { val planned = PartialAggregation(query) assert(planned.isEmpty) } + + test("equi-join is hash-join") { + val x = testData2.as('x) + val y = testData2.as('y) + val join = x.join(y, Inner, Some("x.a".attr === "y.a".attr)).queryExecution.analyzed + val planned = planner.HashJoin(join) + assert(planned.size === 1) + } + + test("multiple-key equi-join is hash-join") { + val x = testData2.as('x) + val y = testData2.as('y) + val join = x.join(y, Inner, + Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)).queryExecution.analyzed + val planned = planner.HashJoin(join) + assert(planned.size === 1) + } } From 2a948e7e1a345ae4e3d89ea24123410819d799d1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 21 May 2014 18:49:12 -0700 Subject: [PATCH 406/641] Configuration documentation updates 1. Add < code > to configuration options 2. List env variables in tabular format to be consistent with other pages. 3. Moved Viewing Spark Properties section up. This is against branch-1.0, but should be cherry picked into master as well. Author: Reynold Xin Closes #851 from rxin/doc-config and squashes the following commits: 28ac0d3 [Reynold Xin] Add to configuration options, and list env variables in a table. (cherry picked from commit 75af8bd3336d09e8c691e54ae9d2358fe1bf3723) Signed-off-by: Reynold Xin --- docs/configuration.md | 194 +++++++++++++++++++++++------------------- 1 file changed, 105 insertions(+), 89 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 462a9d9013d98..4d41c36e38e26 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -47,6 +47,13 @@ Any values specified in the file will be passed on to the application, and merge specified through SparkConf. If the same configuration property exists in both `spark-defaults.conf` and SparkConf, then the latter will take precedence as it is the most application-specific. +## Viewing Spark Properties + +The application web UI at `http://:4040` lists Spark properties in the "Environment" tab. +This is a useful place to check to make sure that your properties have been set correctly. Note +that only values explicitly specified through either `spark-defaults.conf` or SparkConf will +appear. For all other configuration properties, you can assume the default value is used. + ## All Configuration Properties Most of the properties that control internal settings have reasonable default values. However, @@ -55,14 +62,14 @@ there are at least five properties that you will commonly want to control: - + - + - + - + - + @@ -93,9 +89,9 @@ there are at least five properties that you will commonly want to control: @@ -167,8 +163,9 @@ Apart from these, the following properties are also available, and may be useful @@ -182,9 +179,9 @@ Apart from these, the following properties are also available, and may be useful @@ -271,15 +268,17 @@ Apart from these, the following properties are also available, and may be useful @@ -296,7 +295,8 @@ Apart from these, the following properties are also available, and may be useful @@ -304,8 +304,8 @@ Apart from these, the following properties are also available, and may be useful @@ -329,9 +329,10 @@ Apart from these, the following properties are also available, and may be useful @@ -399,9 +400,9 @@ Apart from these, the following properties are also available, and may be useful @@ -416,10 +417,10 @@ Apart from these, the following properties are also available, and may be useful @@ -450,21 +451,36 @@ Apart from these, the following properties are also available, and may be useful @@ -485,10 +501,11 @@ Apart from these, the following properties are also available, and may be useful @@ -523,7 +540,8 @@ Apart from these, the following properties are also available, and may be useful @@ -531,7 +549,10 @@ Apart from these, the following properties are also available, and may be useful @@ -546,15 +567,16 @@ Apart from these, the following properties are also available, and may be useful @@ -589,7 +611,8 @@ Apart from these, the following properties are also available, and may be useful @@ -604,39 +627,40 @@ Apart from these, the following properties are also available, and may be useful @@ -660,8 +684,8 @@ Apart from these, the following properties are also available, and may be useful @@ -719,13 +743,13 @@ Apart from these, the following properties are also available, and may be useful # Environment Variables -Certain Spark settings can be configured through environment variables, which are read from the `conf/spark-env.sh` -script in the directory where Spark is installed (or `conf/spark-env.cmd` on Windows). In Standalone and Mesos modes, -this file can give machine specific information such as hostnames. It is also sourced when running local -Spark applications or submission scripts. +Certain Spark settings can be configured through environment variables, which are read from the +`conf/spark-env.sh` script in the directory where Spark is installed (or `conf/spark-env.cmd` on +Windows). In Standalone and Mesos modes, this file can give machine specific information such as +hostnames. It is also sourced when running local Spark applications or submission scripts. -Note that `conf/spark-env.sh` does not exist by default when Spark is installed. However, you can copy -`conf/spark-env.sh.template` to create it. Make sure you make the copy executable. +Note that `conf/spark-env.sh` does not exist by default when Spark is installed. However, you can +copy `conf/spark-env.sh.template` to create it. Make sure you make the copy executable. The following variables can be set in `spark-env.sh`: @@ -750,12 +774,104 @@ The following variables can be set in `spark-env.sh`:
      Property NameDefaultMeaning
      spark.executor.memoryspark.executor.memory 512m Amount of memory to use per executor process, in the same format as JVM memory strings (e.g. 512m, 2g).
      spark.serializerspark.serializer org.apache.spark.serializer.
      JavaSerializer
      Class to use for serializing objects that will be sent over the network or need to be cached @@ -73,7 +80,7 @@ there are at least five properties that you will commonly want to control:
      spark.kryo.registratorspark.kryo.registrator (none) If you use Kryo serialization, set this class to register your custom classes with Kryo. @@ -83,7 +90,7 @@ there are at least five properties that you will commonly want to control:
      spark.local.dirspark.local.dir /tmp Directory to use for "scratch" space in Spark, including map output files and RDDs that get stored @@ -95,7 +102,7 @@ there are at least five properties that you will commonly want to control:
      spark.cores.maxspark.cores.max (not set) When running on a standalone deploy cluster or a @@ -114,7 +121,7 @@ Apart from these, the following properties are also available, and may be useful - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - + - +
      Property NameDefaultMeaning
      spark.default.parallelismspark.default.parallelism
      • Local mode: number of cores on the local machine
      • @@ -128,7 +135,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.storage.memoryFractionspark.storage.memoryFraction 0.6 Fraction of Java heap to use for Spark's memory cache. This should not be larger than the "old" @@ -137,7 +144,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.shuffle.memoryFractionspark.shuffle.memoryFraction 0.3 Fraction of Java heap to use for aggregation and cogroups during shuffles, if @@ -148,7 +155,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.storage.memoryMapThresholdspark.storage.memoryMapThreshold 8192 Size of a block, in bytes, above which Spark memory maps when reading a block from disk. @@ -157,7 +164,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.tachyonStore.baseDirspark.tachyonStore.baseDir System.getProperty("java.io.tmpdir") Directories of the Tachyon File System that store RDDs. The Tachyon file system's URL is set by spark.tachyonStore.url. @@ -165,14 +172,14 @@ Apart from these, the following properties are also available, and may be useful
      spark.tachyonStore.urlspark.tachyonStore.url tachyon://localhost:19998 The URL of the underlying Tachyon file system in the TachyonStore.
      spark.mesos.coarsespark.mesos.coarse false If set to "true", runs over Mesos clusters in @@ -183,21 +190,21 @@ Apart from these, the following properties are also available, and may be useful
      spark.ui.portspark.ui.port 4040 Port for your application's dashboard, which shows memory and workload data
      spark.ui.retainedStagesspark.ui.retainedStages 1000 How many stages the Spark UI remembers before garbage collecting.
      spark.ui.filtersspark.ui.filters None Comma separated list of filter class names to apply to the Spark web ui. The filter should be a @@ -207,7 +214,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.ui.acls.enablespark.ui.acls.enable false Whether spark web ui acls should are enabled. If enabled, this checks to see if the user has @@ -217,7 +224,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.ui.view.aclsspark.ui.view.acls Empty Comma separated list of users that have view access to the spark web ui. By default only the @@ -225,35 +232,35 @@ Apart from these, the following properties are also available, and may be useful
      spark.ui.killEnabledspark.ui.killEnabled true Allows stages and corresponding jobs to be killed from the web ui.
      spark.shuffle.compressspark.shuffle.compress true Whether to compress map output files. Generally a good idea.
      spark.shuffle.spill.compressspark.shuffle.spill.compress true Whether to compress data spilled during shuffles.
      spark.broadcast.compressspark.broadcast.compress true Whether to compress broadcast variables before sending them. Generally a good idea.
      spark.rdd.compressspark.rdd.compress false Whether to compress serialized RDD partitions (e.g. for StorageLevel.MEMORY_ONLY_SER). @@ -261,7 +268,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.io.compression.codecspark.io.compression.codec org.apache.spark.io.
      LZFCompressionCodec
      The codec used to compress internal data such as RDD partitions and shuffle outputs. By default, Spark provides two @@ -269,14 +276,14 @@ Apart from these, the following properties are also available, and may be useful
      spark.io.compression.snappy.block.sizespark.io.compression.snappy.block.size 32768 Block size (in bytes) used in Snappy compression, in the case when Snappy compression codec is used.
      spark.scheduler.modespark.scheduler.mode FIFO The scheduling mode between @@ -286,14 +293,14 @@ Apart from these, the following properties are also available, and may be useful
      spark.scheduler.revive.intervalspark.scheduler.revive.interval 1000 The interval length for the scheduler to revive the worker resource offers to run tasks. (in milliseconds)
      spark.reducer.maxMbInFlightspark.reducer.maxMbInFlight 48 Maximum size (in megabytes) of map outputs to fetch simultaneously from each reduce task. Since @@ -302,14 +309,14 @@ Apart from these, the following properties are also available, and may be useful
      spark.closure.serializerspark.closure.serializer org.apache.spark.serializer.
      JavaSerializer
      Serializer class to use for closures. Currently only the Java serializer is supported.
      spark.kryo.referenceTrackingspark.kryo.referenceTracking true Whether to track references to the same object when serializing data with Kryo, which is @@ -319,7 +326,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.kryoserializer.buffer.mbspark.kryoserializer.buffer.mb 2 Maximum object size to allow within Kryo (the library needs to create a buffer at least as @@ -328,7 +335,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.serializer.objectStreamResetspark.serializer.objectStreamReset 10000 When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches @@ -339,14 +346,14 @@ Apart from these, the following properties are also available, and may be useful
      spark.broadcast.factoryspark.broadcast.factory org.apache.spark.broadcast.
      HttpBroadcastFactory
      Which broadcast implementation to use.
      spark.locality.waitspark.locality.wait 3000 Number of milliseconds to wait to launch a data-local task before giving up and launching it @@ -358,7 +365,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.locality.wait.processspark.locality.wait.process spark.locality.wait Customize the locality wait for process locality. This affects tasks that attempt to access @@ -366,7 +373,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.locality.wait.nodespark.locality.wait.node spark.locality.wait Customize the locality wait for node locality. For example, you can set this to 0 to skip @@ -374,14 +381,14 @@ Apart from these, the following properties are also available, and may be useful
      spark.locality.wait.rackspark.locality.wait.rack spark.locality.wait Customize the locality wait for rack locality.
      spark.worker.timeoutspark.worker.timeout 60 Number of seconds after which the standalone deploy master considers a worker lost if it @@ -389,7 +396,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.worker.cleanup.enabledspark.worker.cleanup.enabled false Enable periodic cleanup of worker / application directories. Note that this only affects standalone @@ -398,7 +405,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.worker.cleanup.intervalspark.worker.cleanup.interval 1800 (30 minutes) Controls the interval, in seconds, at which the worker cleans up old application work dirs @@ -406,7 +413,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.worker.cleanup.appDataTtlspark.worker.cleanup.appDataTtl 7 * 24 * 3600 (7 days) The number of seconds to retain application work directories on each worker. This is a Time To Live @@ -416,7 +423,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.akka.frameSizespark.akka.frameSize 10 Maximum message size to allow in "control plane" communication (for serialized tasks and task @@ -425,7 +432,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.akka.threadsspark.akka.threads 4 Number of actor threads to use for communication. Can be useful to increase on large clusters @@ -433,49 +440,49 @@ Apart from these, the following properties are also available, and may be useful
      spark.akka.timeoutspark.akka.timeout 100 Communication timeout between Spark nodes, in seconds.
      spark.akka.heartbeat.pausesspark.akka.heartbeat.pauses 600 This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart beat pause in seconds for akka. This can be used to control sensitivity to gc pauses. Tune this in combination of `spark.akka.heartbeat.interval` and `spark.akka.failure-detector.threshold` if you need to.
      spark.akka.failure-detector.thresholdspark.akka.failure-detector.threshold 300.0 This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). This maps to akka's `akka.remote.transport-failure-detector.threshold`. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.heartbeat.interval` if you need to.
      spark.akka.heartbeat.intervalspark.akka.heartbeat.interval 1000 This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using failure detector can be, a sensistive failure detector can help evict rogue executors really quick. However this is usually not the case as gc pauses and network lags are expected in a real spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats between nodes leading to flooding the network with those.
      spark.driver.hostspark.driver.host (local hostname) Hostname or IP address for the driver to listen on.
      spark.driver.portspark.driver.port (random) Port for the driver to listen on.
      spark.cleaner.ttlspark.cleaner.ttl (infinite) Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks generated, etc.). @@ -485,7 +492,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.streaming.blockIntervalspark.streaming.blockInterval 200 Interval (milliseconds) at which data received by Spark Streaming receivers is coalesced @@ -493,7 +500,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.streaming.unpersistspark.streaming.unpersist true Force RDDs generated and persisted by Spark Streaming to be automatically unpersisted from @@ -504,7 +511,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.task.maxFailuresspark.task.maxFailures 4 Number of individual task failures before giving up on the job. @@ -512,7 +519,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.broadcast.blockSizespark.broadcast.blockSize 4096 Size of each piece of a block in kilobytes for TorrentBroadcastFactory. @@ -521,14 +528,14 @@ Apart from these, the following properties are also available, and may be useful
      spark.shuffle.consolidateFilesspark.shuffle.consolidateFiles false If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files can improve filesystem performance for shuffles with large numbers of reduce tasks. It is recommended to set this to "true" when using ext4 or xfs filesystems. On ext3, this option might degrade performance on machines with many (>8) cores due to filesystem limitations.
      spark.shuffle.file.buffer.kbspark.shuffle.file.buffer.kb 100 Size of the in-memory buffer for each shuffle file output stream, in kilobytes. These buffers @@ -536,7 +543,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.shuffle.spillspark.shuffle.spill true If set to "true", limits the amount of memory used during reduces by spilling data out to disk. This spilling @@ -544,56 +551,56 @@ Apart from these, the following properties are also available, and may be useful
      spark.speculationspark.speculation false If set to "true", performs speculative execution of tasks. This means if one or more tasks are running slowly in a stage, they will be re-launched.
      spark.speculation.intervalspark.speculation.interval 100 How often Spark will check for tasks to speculate, in milliseconds.
      spark.speculation.quantilespark.speculation.quantile 0.75 Percentage of tasks which must be complete before speculation is enabled for a particular stage.
      spark.speculation.multiplierspark.speculation.multiplier 1.5 How many times slower a task is than the median to be considered for speculation.
      spark.logConfspark.logConf false Whether to log the supplied SparkConf as INFO at start of spark context.
      spark.eventLog.enabledspark.eventLog.enabled false Whether to log spark events, useful for reconstructing the Web UI after the application has finished.
      spark.eventLog.compressspark.eventLog.compress false Whether to compress logged events, if spark.eventLog.enabled is true.
      spark.eventLog.dirspark.eventLog.dir file:///tmp/spark-events Base directory in which spark events are logged, if spark.eventLog.enabled is true. @@ -602,7 +609,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.deploy.spreadOutspark.deploy.spreadOut true Whether the standalone cluster manager should spread applications out across nodes or try @@ -613,7 +620,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.deploy.defaultCoresspark.deploy.defaultCores (infinite) Default number of cores to give to applications in Spark's standalone mode if they don't @@ -626,14 +633,14 @@ Apart from these, the following properties are also available, and may be useful
      spark.files.overwritespark.files.overwrite false Whether to overwrite files added through SparkContext.addFile() when the target file exists and its contents do not match those of the source.
      spark.files.fetchTimeoutspark.files.fetchTimeout false Communication timeout to use when fetching files added through SparkContext.addFile() from @@ -641,7 +648,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.files.userClassPathFirstspark.files.userClassPathFirst false (Experimental) Whether to give user-added jars precedence over Spark's own jars when @@ -650,7 +657,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.authenticatespark.authenticate false Whether spark authenticates its internal connections. See spark.authenticate.secret if not @@ -658,7 +665,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.authenticate.secretspark.authenticate.secret None Set the secret key used for Spark to authenticate between components. This needs to be set if @@ -666,7 +673,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.core.connection.auth.wait.timeoutspark.core.connection.auth.wait.timeout 30 Number of seconds for the connection to wait for authentication to occur before timing @@ -674,14 +681,14 @@ Apart from these, the following properties are also available, and may be useful
      spark.task.cpusspark.task.cpus 1 Number of cores to allocate for each task.
      spark.executor.extraJavaOptionsspark.executor.extraJavaOptions (none) A string of extra JVM options to pass to executors. For instance, GC settings or other @@ -692,7 +699,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.executor.extraClassPathspark.executor.extraClassPath (none) Extra classpath entries to append to the classpath of executors. This exists primarily @@ -701,7 +708,7 @@ Apart from these, the following properties are also available, and may be useful
      spark.executor.extraLibraryPathspark.executor.extraLibraryPath (none) Set a special library path to use when launching executor JVM's. @@ -710,13 +717,6 @@ Apart from these, the following properties are also available, and may be useful
      -## Viewing Spark Properties - -The application web UI at `http://:4040` lists Spark properties in the "Environment" tab. -This is a useful place to check to make sure that your properties have been set correctly. Note -that only values explicitly specified through either `spark-defaults.conf` or SparkConf will -appear. For all other configuration properties, you can assume the default value is used. - # Environment Variables Certain Spark settings can be configured through environment variables, which are read from the `conf/spark-env.sh` @@ -729,12 +729,28 @@ Note that `conf/spark-env.sh` does not exist by default when Spark is installed. The following variables can be set in `spark-env.sh`: -* `JAVA_HOME`, the location where Java is installed (if it's not on your default `PATH`) -* `PYSPARK_PYTHON`, the Python binary to use for PySpark -* `SPARK_LOCAL_IP`, to configure which IP address of the machine to bind to. -* `SPARK_PUBLIC_DNS`, the hostname your Spark program will advertise to other machines. -* Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), - such as number of cores to use on each machine and maximum memory. + + + + + + + + + + + + + + + + + + + +
      Environment VariableMeaning
      JAVA_HOMELocation where Java is installed (if it's not on your default `PATH`).
      PYSPARK_PYTHONPython binary executable to use for PySpark.
      SPARK_LOCAL_IPIP address of the machine to bind to.
      SPARK_PUBLIC_DNSHostname your Spark program will advertise to other machines.
      + +In addition to the above, there are also options for setting up the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores to use on each machine and maximum memory. Since `spark-env.sh` is a shell script, some of these can be set programmatically -- for example, you might compute `SPARK_LOCAL_IP` by looking up the IP of a specific network interface. From dba314029b4c9d72d7e48a2093b39edd01931f57 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 22 May 2014 01:52:50 -0700 Subject: [PATCH 407/641] [SPARK-1870] Make spark-submit --jars work in yarn-cluster mode. Sent secondary jars to distributed cache of all containers and add the cached jars to classpath before executors start. Tested on a YARN cluster (CDH-5.0). `spark-submit --jars` also works in standalone server and `yarn-client`. Thanks for @andrewor14 for testing! I removed "Doesn't work for drivers in standalone mode with "cluster" deploy mode." from `spark-submit`'s help message, though we haven't tested mesos yet. CC: @dbtsai @sryza Author: Xiangrui Meng Closes #848 from mengxr/yarn-classpath and squashes the following commits: 23e7df4 [Xiangrui Meng] rename spark.jar to __spark__.jar and app.jar to __app__.jar to avoid confliction apped $CWD/ and $CWD/* to the classpath remove unused methods a40f6ed [Xiangrui Meng] standalone -> cluster 65e04ad [Xiangrui Meng] update spark-submit help message and add a comment for yarn-client 11e5354 [Xiangrui Meng] minor changes 3e7e1c4 [Xiangrui Meng] use sparkConf instead of hadoop conf dc3c825 [Xiangrui Meng] add secondary jars to classpath in yarn --- .../spark/deploy/SparkSubmitArguments.scala | 3 +- .../apache/spark/deploy/yarn/ClientBase.scala | 69 +++++-------------- .../cluster/YarnClientSchedulerBackend.scala | 2 +- 3 files changed, 19 insertions(+), 55 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 264d4544cd31c..0cc05fb95aef0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -326,8 +326,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | --class CLASS_NAME Your application's main class (for Java / Scala apps). | --name NAME A name of your application. | --jars JARS Comma-separated list of local jars to include on the driver - | and executor classpaths. Doesn't work for drivers in - | standalone mode with "cluster" deploy mode. + | and executor classpaths. | --py-files PY_FILES Comma-separated list of .zip or .egg files to place on the | PYTHONPATH for Python apps. | --files FILES Comma-separated list of files to be placed in the working 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 27a518ccda459..aeb3f0062df3b 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 @@ -44,7 +44,7 @@ import org.apache.spark.{Logging, SparkConf, SparkContext} * Client submits an application to the YARN ResourceManager. * * Depending on the deployment mode this will launch one of two application master classes: - * 1. In standalone mode, it will launch an [[org.apache.spark.deploy.yarn.ApplicationMaster]] + * 1. In cluster mode, it will launch an [[org.apache.spark.deploy.yarn.ApplicationMaster]] * which launches a driver program inside of the cluster. * 2. In client mode, it will launch an [[org.apache.spark.deploy.yarn.ExecutorLauncher]] to * request executors on behalf of a driver running outside of the cluster. @@ -220,10 +220,11 @@ trait ClientBase extends Logging { } } + var cachedSecondaryJarLinks = ListBuffer.empty[String] val fileLists = List( (args.addJars, LocalResourceType.FILE, true), (args.files, LocalResourceType.FILE, false), (args.archives, LocalResourceType.ARCHIVE, false) ) - fileLists.foreach { case (flist, resType, appMasterOnly) => + fileLists.foreach { case (flist, resType, addToClasspath) => if (flist != null && !flist.isEmpty()) { flist.split(',').foreach { case file: String => val localURI = new URI(file.trim()) @@ -232,11 +233,15 @@ trait ClientBase extends Logging { val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) val destPath = copyRemoteFile(dst, localPath, replication) distCacheMgr.addResource(fs, conf, destPath, localResources, resType, - linkname, statCache, appMasterOnly) + linkname, statCache) + if (addToClasspath) { + cachedSecondaryJarLinks += linkname + } } } } } + sparkConf.set(ClientBase.CONF_SPARK_YARN_SECONDARY_JARS, cachedSecondaryJarLinks.mkString(",")) UserGroupInformation.getCurrentUser().addCredentials(credentials) localResources @@ -374,11 +379,12 @@ trait ClientBase extends Logging { } object ClientBase { - val SPARK_JAR: String = "spark.jar" - val APP_JAR: String = "app.jar" + val SPARK_JAR: String = "__spark__.jar" + val APP_JAR: String = "__app__.jar" val LOG4J_PROP: String = "log4j.properties" val LOG4J_CONF_ENV_KEY: String = "SPARK_LOG4J_CONF" val LOCAL_SCHEME = "local" + val CONF_SPARK_YARN_SECONDARY_JARS = "spark.yarn.secondary.jars" def getSparkJar = sys.env.get("SPARK_JAR").getOrElse(SparkContext.jarOfClass(this.getClass).head) @@ -479,66 +485,25 @@ object ClientBase { extraClassPath.foreach(addClasspathEntry) - addClasspathEntry(Environment.PWD.$()) + val cachedSecondaryJarLinks = + sparkConf.getOption(CONF_SPARK_YARN_SECONDARY_JARS).getOrElse("").split(",") // Normally the users app.jar is last in case conflicts with spark jars if (sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean) { addPwdClasspathEntry(APP_JAR) + cachedSecondaryJarLinks.foreach(addPwdClasspathEntry) addPwdClasspathEntry(SPARK_JAR) ClientBase.populateHadoopClasspath(conf, env) } else { addPwdClasspathEntry(SPARK_JAR) ClientBase.populateHadoopClasspath(conf, env) addPwdClasspathEntry(APP_JAR) + cachedSecondaryJarLinks.foreach(addPwdClasspathEntry) } + // Append all class files and jar files under the working directory to the classpath. + addClasspathEntry(Environment.PWD.$()) addPwdClasspathEntry("*") } - /** - * Adds the user jars which have local: URIs (or alternate names, such as APP_JAR) explicitly - * to the classpath. - */ - private def addUserClasspath(args: ClientArguments, env: HashMap[String, String]) = { - if (args != null) { - addClasspathEntry(args.userJar, APP_JAR, env) - } - - if (args != null && args.addJars != null) { - args.addJars.split(",").foreach { case file: String => - addClasspathEntry(file, null, env) - } - } - } - - /** - * Adds the given path to the classpath, handling "local:" URIs correctly. - * - * If an alternate name for the file is given, and it's not a "local:" file, the alternate - * name will be added to the classpath (relative to the job's work directory). - * - * If not a "local:" file and no alternate name, the environment is not modified. - * - * @param path Path to add to classpath (optional). - * @param fileName Alternate name for the file (optional). - * @param env Map holding the environment variables. - */ - private def addClasspathEntry(path: String, fileName: String, - env: HashMap[String, String]) : Unit = { - if (path != null) { - scala.util.control.Exception.ignoring(classOf[URISyntaxException]) { - val localPath = getLocalPath(path) - if (localPath != null) { - YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, localPath, - File.pathSeparator) - return - } - } - } - if (fileName != null) { - YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, - Environment.PWD.$() + Path.SEPARATOR + fileName, File.pathSeparator); - } - } - /** * Returns the local path if the URI is a "local:" URI, or null otherwise. */ 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 0ac162538fc4b..e01ed5a57d697 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 @@ -52,7 +52,7 @@ private[spark] class YarnClientSchedulerBackend( val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ( "--class", "notused", - "--jar", null, + "--jar", null, // The primary jar will be added dynamically in SparkContext. "--args", hostport, "--am-class", classOf[ExecutorLauncher].getName ) From f9f5fd5f4e81828a3e0c391892e0f28751568843 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 22 May 2014 15:11:05 -0700 Subject: [PATCH 408/641] Fix UISuite unit test that fails under Jenkins contention Due to perhaps zombie processes on Jenkins, it seems that at least 10 Spark ports are in use. It also doesn't matter that the port increases when used, it could in fact go down -- the only part that matters is that it selects a different port rather than failing to bind. Changed test to match this. Thanks to @andrewor14 for helping diagnose this. Author: Aaron Davidson Closes #857 from aarondav/tiny and squashes the following commits: c199ec8 [Aaron Davidson] Fix UISuite unit test that fails under Jenkins contention --- core/src/test/scala/org/apache/spark/ui/UISuite.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index fff8020ade6d1..038746d2eda4b 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -94,7 +94,7 @@ class UISuite extends FunSuite { } } - test("jetty port increases under contention") { + test("jetty selects different port under contention") { val startPort = 4040 val server = new Server(startPort) @@ -110,8 +110,9 @@ class UISuite extends FunSuite { // Allow some wiggle room in case ports on the machine are under contention val boundPort1 = serverInfo1.boundPort val boundPort2 = serverInfo2.boundPort - assert(boundPort1 > startPort && boundPort1 < startPort + 10) - assert(boundPort2 > boundPort1 && boundPort2 < boundPort1 + 10) + assert(boundPort1 != startPort) + assert(boundPort2 != startPort) + assert(boundPort1 != boundPort2) } test("jetty binds to port 0 correctly") { From 8edbee7d1b4afc192d97ba192a5526affc464205 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 22 May 2014 20:25:41 -0700 Subject: [PATCH 409/641] [SPARK-1897] Respect spark.jars (and --jars) in spark-shell Spark shell currently overwrites `spark.jars` with `ADD_JARS`. In all modes except yarn-cluster, this means the `--jar` flag passed to `bin/spark-shell` is also discarded. However, in the [docs](http://people.apache.org/~pwendell/spark-1.0.0-rc7-docs/scala-programming-guide.html#initializing-spark), we explicitly tell the users to add the jars this way. Author: Andrew Or Closes #849 from andrewor14/shell-jars and squashes the following commits: 928a7e6 [Andrew Or] ',' -> "," (minor) afc357c [Andrew Or] Handle spark.jars == "" in SparkILoop, not SparkSubmit c6da113 [Andrew Or] Do not set spark.jars to "" d8549f7 [Andrew Or] Respect spark.jars and --jars in spark-shell --- .../src/main/scala/org/apache/spark/repl/SparkILoop.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 296da740687ec..55684e94bd614 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -993,7 +993,13 @@ object SparkILoop { implicit def loopToInterpreter(repl: SparkILoop): SparkIMain = repl.intp private def echo(msg: String) = Console println msg - def getAddedJars: Array[String] = Option(System.getenv("ADD_JARS")).map(_.split(',')).getOrElse(new Array[String](0)) + def getAddedJars: Array[String] = { + val envJars = sys.env.get("ADD_JARS") + val propJars = sys.props.get("spark.jars").flatMap { p => + if (p == "") None else Some(p) + } + propJars.orElse(envJars).map(_.split(",")).getOrElse(Array.empty) + } // Designed primarily for use by test code: take a String with a // bunch of code, and prints out a transcript of what it would look From cce77457e00aa5f1f4db3d50454cf257efb156ed Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 22 May 2014 20:32:27 -0700 Subject: [PATCH 410/641] [SPARK-1896] Respect spark.master (and --master) before MASTER in spark-shell The hierarchy for configuring the Spark master in the shell is as follows: ``` MASTER > --master > spark.master (spark-defaults.conf) ``` This is inconsistent with the way we run normal applications, which is: ``` --master > spark.master (spark-defaults.conf) > MASTER ``` I was trying to run a shell locally on a standalone cluster launched through the ec2 scripts, which automatically set `MASTER` in spark-env.sh. It was surprising to me that `--master` didn't take effect, considering that this is the way we tell users to set their masters [here](http://people.apache.org/~pwendell/spark-1.0.0-rc7-docs/scala-programming-guide.html#initializing-spark). Author: Andrew Or Closes #846 from andrewor14/shell-master and squashes the following commits: 2cb81c9 [Andrew Or] Respect spark.master before MASTER in REPL --- repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 55684e94bd614..5f34362ccd973 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -962,11 +962,10 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, private def getMaster(): String = { val master = this.master match { case Some(m) => m - case None => { + case None => val envMaster = sys.env.get("MASTER") val propMaster = sys.props.get("spark.master") - envMaster.orElse(propMaster).getOrElse("local[*]") - } + propMaster.orElse(envMaster).getOrElse("local[*]") } master } From b2bdd0e505f1ae3d39c46139f17bd43779ece635 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 22 May 2014 20:48:55 -0700 Subject: [PATCH 411/641] Updated scripts for auditing releases - Added script to automatically generate change list CHANGES.txt - Added test for verifying linking against maven distributions of `spark-sql` and `spark-hive` - Added SBT projects for testing functionality of `spark-sql` and `spark-hive` - Fixed issues in existing tests that might have come up because of changes in Spark 1.0 Author: Tathagata Das Closes #844 from tdas/update-dev-scripts and squashes the following commits: 25090ba [Tathagata Das] Added missing license e2e20b3 [Tathagata Das] Updated tests for auditing releases. --- dev/audit-release/audit_release.py | 11 +- .../src/main/java/SimpleApp.java | 1 + .../src/main/scala/SparkApp.scala | 2 +- dev/audit-release/sbt_app_hive/build.sbt | 29 +++ dev/audit-release/sbt_app_hive/data.txt | 9 + .../src/main/resources/hive-site.xml | 213 ++++++++++++++++++ .../sbt_app_hive/src/main/scala/HiveApp.scala | 57 +++++ dev/audit-release/sbt_app_sql/build.sbt | 29 +++ .../sbt_app_sql/src/main/scala/SqlApp.scala | 57 +++++ .../src/main/scala/StreamingApp.scala | 1 - dev/create-release/generate-changelist.py | 144 ++++++++++++ 11 files changed, 547 insertions(+), 6 deletions(-) create mode 100644 dev/audit-release/sbt_app_hive/build.sbt create mode 100644 dev/audit-release/sbt_app_hive/data.txt create mode 100644 dev/audit-release/sbt_app_hive/src/main/resources/hive-site.xml create mode 100644 dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala create mode 100644 dev/audit-release/sbt_app_sql/build.sbt create mode 100644 dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala create mode 100755 dev/create-release/generate-changelist.py diff --git a/dev/audit-release/audit_release.py b/dev/audit-release/audit_release.py index 4a816d4101e57..8c7573b91f688 100755 --- a/dev/audit-release/audit_release.py +++ b/dev/audit-release/audit_release.py @@ -93,9 +93,12 @@ def get_url(url): # For each of these modules, we'll test an 'empty' application in sbt and # maven that links against them. This will catch issues with messed up # dependencies within those projects. -modules = ["spark-core", "spark-bagel", "spark-mllib", "spark-streaming", "spark-repl", - "spark-graphx", "spark-streaming-flume", "spark-streaming-kafka", - "spark-streaming-mqtt", "spark-streaming-twitter", "spark-streaming-zeromq"] +modules = [ + "spark-core", "spark-bagel", "spark-mllib", "spark-streaming", "spark-repl", + "spark-graphx", "spark-streaming-flume", "spark-streaming-kafka", + "spark-streaming-mqtt", "spark-streaming-twitter", "spark-streaming-zeromq", + "spark-catalyst", "spark-sql", "spark-hive" +] modules = map(lambda m: "%s_%s" % (m, SCALA_BINARY_VERSION), modules) # Check for directories that might interfere with tests @@ -122,7 +125,7 @@ def ensure_path_not_present(x): os.chdir(original_dir) # SBT application tests -for app in ["sbt_app_core", "sbt_app_graphx", "sbt_app_streaming"]: +for app in ["sbt_app_core", "sbt_app_graphx", "sbt_app_streaming", "sbt_app_sql", "sbt_app_hive"]: os.chdir(app) ret = run_cmd("sbt clean run", exit_on_failure=False) test(ret == 0, "sbt application (%s)" % app) diff --git a/dev/audit-release/maven_app_core/src/main/java/SimpleApp.java b/dev/audit-release/maven_app_core/src/main/java/SimpleApp.java index 6b65dda39b1a2..5217689e7c092 100644 --- a/dev/audit-release/maven_app_core/src/main/java/SimpleApp.java +++ b/dev/audit-release/maven_app_core/src/main/java/SimpleApp.java @@ -37,5 +37,6 @@ public static void main(String[] args) { System.exit(-1); } System.out.println("Test succeeded"); + sc.stop(); } } diff --git a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala index a89b0d7d38bf1..77bbd167b199a 100644 --- a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala +++ b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala @@ -19,6 +19,7 @@ package main.scala import scala.util.Try +import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ @@ -30,7 +31,6 @@ object SimpleApp { } val logFile = "input.txt" val sc = new SparkContext(conf) - SparkContext.jarOfClass(this.getClass).foreach(sc.addJar) val logData = sc.textFile(logFile, 2).cache() val numAs = logData.filter(line => line.contains("a")).count() val numBs = logData.filter(line => line.contains("b")).count() diff --git a/dev/audit-release/sbt_app_hive/build.sbt b/dev/audit-release/sbt_app_hive/build.sbt new file mode 100644 index 0000000000000..7ac1be729c561 --- /dev/null +++ b/dev/audit-release/sbt_app_hive/build.sbt @@ -0,0 +1,29 @@ +// +// 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. +// + +name := "Simple Project" + +version := "1.0" + +scalaVersion := System.getenv.get("SCALA_VERSION") + +libraryDependencies += "org.apache.spark" %% "spark-hive" % System.getenv.get("SPARK_VERSION") + +resolvers ++= Seq( + "Spark Release Repository" at System.getenv.get("SPARK_RELEASE_REPOSITORY"), + "Akka Repository" at "http://repo.akka.io/releases/", + "Spray Repository" at "http://repo.spray.cc/") diff --git a/dev/audit-release/sbt_app_hive/data.txt b/dev/audit-release/sbt_app_hive/data.txt new file mode 100644 index 0000000000000..0229e67f51e01 --- /dev/null +++ b/dev/audit-release/sbt_app_hive/data.txt @@ -0,0 +1,9 @@ +0val_0 +1val_1 +2val_2 +3val_3 +4val_4 +5val_5 +6val_6 +7val_7 +9val_9 diff --git a/dev/audit-release/sbt_app_hive/src/main/resources/hive-site.xml b/dev/audit-release/sbt_app_hive/src/main/resources/hive-site.xml new file mode 100644 index 0000000000000..93b835813d535 --- /dev/null +++ b/dev/audit-release/sbt_app_hive/src/main/resources/hive-site.xml @@ -0,0 +1,213 @@ + + + + + + + + + + + + + + + + + + build.dir + ${user.dir}/build + + + + build.dir.hive + ${build.dir}/hive + + + + hadoop.tmp.dir + ${build.dir.hive}/test/hadoop-${user.name} + A base for other temporary directories. + + + + + + hive.exec.scratchdir + ${build.dir}/scratchdir + Scratch space for Hive jobs + + + + hive.exec.local.scratchdir + ${build.dir}/localscratchdir/ + Local scratch space for Hive jobs + + + + javax.jdo.option.ConnectionURL + + jdbc:derby:;databaseName=../build/test/junit_metastore_db;create=true + + + + javax.jdo.option.ConnectionDriverName + org.apache.derby.jdbc.EmbeddedDriver + + + + javax.jdo.option.ConnectionUserName + APP + + + + javax.jdo.option.ConnectionPassword + mine + + + + + hive.metastore.warehouse.dir + ${test.warehouse.dir} + + + + + hive.metastore.metadb.dir + ${build.dir}/test/data/metadb/ + + Required by metastore server or if the uris argument below is not supplied + + + + + test.log.dir + ${build.dir}/test/logs + + + + + test.src.dir + ${build.dir}/src/test + + + + + + + hive.jar.path + ${build.dir.hive}/ql/hive-exec-${version}.jar + + + + + hive.metastore.rawstore.impl + org.apache.hadoop.hive.metastore.ObjectStore + Name of the class that implements org.apache.hadoop.hive.metastore.rawstore interface. This class is used to store and retrieval of raw metadata objects such as table, database + + + + hive.querylog.location + ${build.dir}/tmp + Location of the structured hive logs + + + + + + hive.task.progress + false + Track progress of a task + + + + hive.support.concurrency + false + Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks. + + + + fs.pfile.impl + org.apache.hadoop.fs.ProxyLocalFileSystem + A proxy for local file system used for cross file system testing + + + + hive.exec.mode.local.auto + false + + Let hive determine whether to run in local mode automatically + Disabling this for tests so that minimr is not affected + + + + + hive.auto.convert.join + false + Whether Hive enable the optimization about converting common join into mapjoin based on the input file size + + + + hive.ignore.mapjoin.hint + false + Whether Hive ignores the mapjoin hint + + + + hive.input.format + org.apache.hadoop.hive.ql.io.CombineHiveInputFormat + The default input format, if it is not specified, the system assigns it. It is set to HiveInputFormat for hadoop versions 17, 18 and 19, whereas it is set to CombineHiveInputFormat for hadoop 20. The user can always overwrite it - if there is a bug in CombineHiveInputFormat, it can always be manually set to HiveInputFormat. + + + + hive.default.rcfile.serde + org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + The default SerDe hive will use for the rcfile format + + + diff --git a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala new file mode 100644 index 0000000000000..7257d17d10116 --- /dev/null +++ b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala @@ -0,0 +1,57 @@ +/* + * 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 main.scala + +import scala.collection.mutable.{ListBuffer, Queue} + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.hive.LocalHiveContext + +case class Person(name: String, age: Int) + +object SparkSqlExample { + + def main(args: Array[String]) { + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple Sql App").setMaster(master) + case None => new SparkConf().setAppName("Simple Sql App") + } + val sc = new SparkContext(conf) + val hiveContext = new LocalHiveContext(sc) + + import hiveContext._ + hql("DROP TABLE IF EXISTS src") + hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + hql("LOAD DATA LOCAL INPATH 'data.txt' INTO TABLE src") + val results = hql("FROM src SELECT key, value WHERE key >= 0 AND KEY < 5").collect() + results.foreach(println) + + def test(f: => Boolean, failureMsg: String) = { + if (!f) { + println(failureMsg) + System.exit(-1) + } + } + + test(results.size == 5, "Unexpected number of selected elements: " + results) + println("Test succeeded") + sc.stop() + } +} diff --git a/dev/audit-release/sbt_app_sql/build.sbt b/dev/audit-release/sbt_app_sql/build.sbt new file mode 100644 index 0000000000000..6e0ad3b4b2960 --- /dev/null +++ b/dev/audit-release/sbt_app_sql/build.sbt @@ -0,0 +1,29 @@ +// +// 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. +// + +name := "Simple Project" + +version := "1.0" + +scalaVersion := System.getenv.get("SCALA_VERSION") + +libraryDependencies += "org.apache.spark" %% "spark-sql" % System.getenv.get("SPARK_VERSION") + +resolvers ++= Seq( + "Spark Release Repository" at System.getenv.get("SPARK_RELEASE_REPOSITORY"), + "Akka Repository" at "http://repo.akka.io/releases/", + "Spray Repository" at "http://repo.spray.cc/") diff --git a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala new file mode 100644 index 0000000000000..50af90c213b5a --- /dev/null +++ b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala @@ -0,0 +1,57 @@ +/* + * 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 main.scala + +import scala.collection.mutable.{ListBuffer, Queue} + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext + +case class Person(name: String, age: Int) + +object SparkSqlExample { + + def main(args: Array[String]) { + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple Sql App").setMaster(master) + case None => new SparkConf().setAppName("Simple Sql App") + } + val sc = new SparkContext(conf) + val sqlContext = new SQLContext(sc) + + import sqlContext._ + val people = sc.makeRDD(1 to 100, 10).map(x => Person(s"Name$x", x)) + people.registerAsTable("people") + val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + val teenagerNames = teenagers.map(t => "Name: " + t(0)).collect() + teenagerNames.foreach(println) + + def test(f: => Boolean, failureMsg: String) = { + if (!f) { + println(failureMsg) + System.exit(-1) + } + } + + test(teenagerNames.size == 7, "Unexpected number of selected elements: " + teenagerNames) + println("Test succeeded") + sc.stop() + } +} diff --git a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala index a1d8971abe9a4..58a662bd9b2e8 100644 --- a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala +++ b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala @@ -32,7 +32,6 @@ object SparkStreamingExample { case None => new SparkConf().setAppName("Simple Streaming App") } val ssc = new StreamingContext(conf, Seconds(1)) - SparkContext.jarOfClass(this.getClass).foreach(ssc.sparkContext.addJar) val seen = ListBuffer[RDD[Int]]() val rdd1 = ssc.sparkContext.makeRDD(1 to 100, 10) diff --git a/dev/create-release/generate-changelist.py b/dev/create-release/generate-changelist.py new file mode 100755 index 0000000000000..13b744ec1b37e --- /dev/null +++ b/dev/create-release/generate-changelist.py @@ -0,0 +1,144 @@ +#!/usr/bin/python + +# +# 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. +# +# Creates CHANGES.txt from git history. +# +# Usage: +# First set the new release version and old CHANGES.txt version in this file. +# Make sure you have SPARK_HOME set. +# $ python generate-changelist.py + + +import os +import sys +import subprocess +import time +import traceback + +SPARK_HOME = os.environ["SPARK_HOME"] +NEW_RELEASE_VERSION = "1.0.0" +PREV_RELEASE_GIT_TAG = "v0.9.1" + +CHANGELIST = "CHANGES.txt" +OLD_CHANGELIST = "%s.old" % (CHANGELIST) +NEW_CHANGELIST = "%s.new" % (CHANGELIST) +TMP_CHANGELIST = "%s.tmp" % (CHANGELIST) + +# date before first PR in TLP Spark repo +SPARK_REPO_CHANGE_DATE1 = time.strptime("2014-02-26", "%Y-%m-%d") +# date after last PR in incubator Spark repo +SPARK_REPO_CHANGE_DATE2 = time.strptime("2014-03-01", "%Y-%m-%d") +# Threshold PR number that differentiates PRs to TLP +# and incubator repos +SPARK_REPO_PR_NUM_THRESH = 200 + +LOG_FILE_NAME = "changes_%s" % time.strftime("%h_%m_%Y_%I_%M_%S") +LOG_FILE = open(LOG_FILE_NAME, 'w') + +def run_cmd(cmd): + try: + print >> LOG_FILE, "Running command: %s" % cmd + output = subprocess.check_output(cmd, shell=True, stderr=LOG_FILE) + print >> LOG_FILE, "Output: %s" % output + return output + except: + traceback.print_exc() + cleanup() + sys.exit(1) + +def append_to_changelist(string): + with open(TMP_CHANGELIST, "a") as f: + print >> f, string + +def cleanup(ask = True): + if ask == True: + print "OK to delete temporary and log files? (y/N): " + response = raw_input() + if ask == False or (ask == True and response == "y"): + if os.path.isfile(TMP_CHANGELIST): + os.remove(TMP_CHANGELIST) + if os.path.isfile(OLD_CHANGELIST): + os.remove(OLD_CHANGELIST) + LOG_FILE.close() + os.remove(LOG_FILE_NAME) + +print "Generating new %s for Spark release %s" % (CHANGELIST, NEW_RELEASE_VERSION) +os.chdir(SPARK_HOME) +if os.path.isfile(TMP_CHANGELIST): + os.remove(TMP_CHANGELIST) +if os.path.isfile(OLD_CHANGELIST): + os.remove(OLD_CHANGELIST) + +append_to_changelist("Spark Change Log") +append_to_changelist("----------------") +append_to_changelist("") +append_to_changelist("Release %s" % NEW_RELEASE_VERSION) +append_to_changelist("") + +print "Getting commits between tag %s and HEAD" % PREV_RELEASE_GIT_TAG +hashes = run_cmd("git log %s..HEAD --pretty='%%h'" % PREV_RELEASE_GIT_TAG).split() + +print "Getting details of %s commits" % len(hashes) +for h in hashes: + date = run_cmd("git log %s -1 --pretty='%%ad' --date=iso | head -1" % h).strip() + subject = run_cmd("git log %s -1 --pretty='%%s' | head -1" % h).strip() + body = run_cmd("git log %s -1 --pretty='%%b'" % h) + committer = run_cmd("git log %s -1 --pretty='%%cn <%%ce>' | head -1" % h).strip() + body_lines = body.split("\n") + + if "Merge pull" in subject: + ## Parse old format commit message + append_to_changelist(" %s %s" % (h, date)) + append_to_changelist(" %s" % subject) + append_to_changelist(" [%s]" % body_lines[0]) + append_to_changelist("") + + elif "maven-release" not in subject: + ## Parse new format commit message + # Get authors from commit message, committer otherwise + authors = [committer] + if "Author:" in body: + authors = [line.split(":")[1].strip() for line in body_lines if "Author:" in line] + + # Generate GitHub PR URL for easy access if possible + github_url = "" + if "Closes #" in body: + pr_num = [line.split()[1].lstrip("#") for line in body_lines if "Closes #" in line][0] + github_url = "github.com/apache/spark/pull/%s" % pr_num + day = time.strptime(date.split()[0], "%Y-%m-%d") + if day < SPARK_REPO_CHANGE_DATE1 or (day < SPARK_REPO_CHANGE_DATE2 and pr_num < SPARK_REPO_PR_NUM_THRESH): + github_url = "github.com/apache/incubator-spark/pull/%s" % pr_num + + append_to_changelist(" %s" % subject) + append_to_changelist(" %s" % ', '.join(authors)) + # for author in authors: + # append_to_changelist(" %s" % author) + append_to_changelist(" %s" % date) + if len(github_url) > 0: + append_to_changelist(" Commit: %s, %s" % (h, github_url)) + else: + append_to_changelist(" Commit: %s" % h) + append_to_changelist("") + +# Append old change list +print "Appending changelist from tag %s" % PREV_RELEASE_GIT_TAG +run_cmd("git show %s:%s | tail -n +3 >> %s" % (PREV_RELEASE_GIT_TAG, CHANGELIST, TMP_CHANGELIST)) +run_cmd("cp %s %s" % (TMP_CHANGELIST, NEW_CHANGELIST)) +print "New change list generated as %s" % NEW_CHANGELIST +cleanup(False) + From a08262d8769808dd3a8ee1b1e80fbf6ac13a557c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?baishuo=28=E7=99=BD=E7=A1=95=29?= Date: Fri, 23 May 2014 13:02:40 -0700 Subject: [PATCH 412/641] Update LBFGSSuite.scala MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit the same reason as https://github.com/apache/spark/pull/588 Author: baishuo(白硕) Closes #815 from baishuo/master and squashes the following commits: 6876c1e [baishuo(白硕)] Update LBFGSSuite.scala --- .../org/apache/spark/mllib/optimization/LBFGSSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala index 6af1b502eb4dd..820eca9b1bf65 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala @@ -43,7 +43,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with ShouldMatchers { // Add an extra variable consisting of all 1.0's for the intercept. val testData = GradientDescentSuite.generateGDInput(A, B, nPoints, 42) val data = testData.map { case LabeledPoint(label, features) => - label -> Vectors.dense(1.0, features.toArray: _*) + label -> Vectors.dense(1.0 +: features.toArray) } lazy val dataRDD = sc.parallelize(data, 2).cache() @@ -55,7 +55,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with ShouldMatchers { test("LBFGS loss should be decreasing and match the result of Gradient Descent.") { val regParam = 0 - val initialWeightsWithIntercept = Vectors.dense(1.0, initialWeights: _*) + val initialWeightsWithIntercept = Vectors.dense(1.0 +: initialWeights.toArray) val convergenceTol = 1e-12 val maxNumIterations = 10 From 5081a0a9d47ca31900ea4de570de2cbb0e063105 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 24 May 2014 18:01:49 -0700 Subject: [PATCH 413/641] [SPARK-1900 / 1918] PySpark on YARN is broken If I run the following on a YARN cluster ``` bin/spark-submit sheep.py --master yarn-client ``` it fails because of a mismatch in paths: `spark-submit` thinks that `sheep.py` resides on HDFS, and balks when it can't find the file there. A natural workaround is to add the `file:` prefix to the file: ``` bin/spark-submit file:/path/to/sheep.py --master yarn-client ``` However, this also fails. This time it is because python does not understand URI schemes. This PR fixes this by automatically resolving all paths passed as command line argument to `spark-submit` properly. This has the added benefit of keeping file and jar paths consistent across different cluster modes. For python, we strip the URI scheme before we actually try to run it. Much of the code is originally written by @mengxr. Tested on YARN cluster. More tests pending. Author: Andrew Or Closes #853 from andrewor14/submit-paths and squashes the following commits: 0bb097a [Andrew Or] Format path correctly before adding it to PYTHONPATH 323b45c [Andrew Or] Include --py-files on PYTHONPATH for pyspark shell 3c36587 [Andrew Or] Improve error messages (minor) 854aa6a [Andrew Or] Guard against NPE if user gives pathological paths 6638a6b [Andrew Or] Fix spark-shell jar paths after #849 went in 3bb0359 [Andrew Or] Update more comments (minor) 2a1f8a0 [Andrew Or] Update comments (minor) 6af2c77 [Andrew Or] Merge branch 'master' of github.com:apache/spark into submit-paths a68c4d1 [Andrew Or] Handle Windows python file path correctly 427a250 [Andrew Or] Resolve paths properly for Windows a591a4a [Andrew Or] Update tests for resolving URIs 6c8621c [Andrew Or] Move resolveURIs to Utils db8255e [Andrew Or] Merge branch 'master' of github.com:apache/spark into submit-paths f542dce [Andrew Or] Fix outdated tests 691c4ce [Andrew Or] Ignore special primary resource names 5342ac7 [Andrew Or] Add missing space in error message 02f77f3 [Andrew Or] Resolve command line arguments to spark-submit properly --- .../apache/spark/deploy/PythonRunner.scala | 60 ++++++++++++++-- .../org/apache/spark/deploy/SparkSubmit.scala | 25 ++++--- .../spark/deploy/SparkSubmitArguments.scala | 33 ++++++--- .../scala/org/apache/spark/util/Utils.scala | 71 ++++++++++++++++++- .../spark/deploy/PythonRunnerSuite.scala | 61 ++++++++++++++++ .../spark/deploy/SparkSubmitSuite.scala | 41 ++++++----- .../org/apache/spark/util/UtilsSuite.scala | 66 ++++++++++++++++- python/pyspark/context.py | 8 ++- .../org/apache/spark/repl/SparkILoop.scala | 5 +- 9 files changed, 323 insertions(+), 47 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/deploy/PythonRunnerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala index 2dfa02bd26f13..0d6751f3fa6d2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -17,10 +17,13 @@ package org.apache.spark.deploy +import java.net.URI + import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import org.apache.spark.api.python.{PythonUtils, RedirectThread} +import org.apache.spark.util.Utils /** * A main class used by spark-submit to launch Python applications. It executes python as a @@ -28,12 +31,15 @@ import org.apache.spark.api.python.{PythonUtils, RedirectThread} */ object PythonRunner { def main(args: Array[String]) { - val primaryResource = args(0) + val pythonFile = args(0) val pyFiles = args(1) val otherArgs = args.slice(2, args.length) - val pythonExec = sys.env.get("PYSPARK_PYTHON").getOrElse("python") // TODO: get this from conf + // Format python file paths before adding them to the PYTHONPATH + val formattedPythonFile = formatPath(pythonFile) + val formattedPyFiles = formatPaths(pyFiles) + // Launch a Py4J gateway server for the process to connect to; this will let it see our // Java system properties and such val gatewayServer = new py4j.GatewayServer(null, 0) @@ -42,13 +48,13 @@ object PythonRunner { // Build up a PYTHONPATH that includes the Spark assembly JAR (where this class is), the // python directories in SPARK_HOME (if set), and any files in the pyFiles argument val pathElements = new ArrayBuffer[String] - pathElements ++= Option(pyFiles).getOrElse("").split(",") + pathElements ++= formattedPyFiles pathElements += PythonUtils.sparkPythonPath pathElements += sys.env.getOrElse("PYTHONPATH", "") val pythonPath = PythonUtils.mergePythonPaths(pathElements: _*) // Launch Python process - val builder = new ProcessBuilder(Seq(pythonExec, "-u", primaryResource) ++ otherArgs) + val builder = new ProcessBuilder(Seq(pythonExec, "-u", formattedPythonFile) ++ otherArgs) val env = builder.environment() env.put("PYTHONPATH", pythonPath) env.put("PYSPARK_GATEWAY_PORT", "" + gatewayServer.getListeningPort) @@ -59,4 +65,50 @@ object PythonRunner { System.exit(process.waitFor()) } + + /** + * Format the python file path so that it can be added to the PYTHONPATH correctly. + * + * Python does not understand URI schemes in paths. Before adding python files to the + * PYTHONPATH, we need to extract the path from the URI. This is safe to do because we + * currently only support local python files. + */ + def formatPath(path: String, testWindows: Boolean = false): String = { + if (Utils.nonLocalPaths(path, testWindows).nonEmpty) { + throw new IllegalArgumentException("Launching Python applications through " + + s"spark-submit is currently only supported for local files: $path") + } + val windows = Utils.isWindows || testWindows + var formattedPath = if (windows) Utils.formatWindowsPath(path) else path + + // Strip the URI scheme from the path + formattedPath = + new URI(formattedPath).getScheme match { + case Utils.windowsDrive(d) if windows => formattedPath + case null => formattedPath + case _ => new URI(formattedPath).getPath + } + + // Guard against malformed paths potentially throwing NPE + if (formattedPath == null) { + throw new IllegalArgumentException(s"Python file path is malformed: $path") + } + + // In Windows, the drive should not be prefixed with "/" + // For instance, python does not understand "/C:/path/to/sheep.py" + formattedPath = if (windows) formattedPath.stripPrefix("/") else formattedPath + formattedPath + } + + /** + * Format each python file path in the comma-delimited list of paths, so it can be + * added to the PYTHONPATH correctly. + */ + def formatPaths(paths: String, testWindows: Boolean = false): Array[String] = { + Option(paths).getOrElse("") + .split(",") + .filter(_.nonEmpty) + .map { p => formatPath(p, testWindows) } + } + } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index c54331c00fab8..7e9a9344e61f9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -136,9 +136,9 @@ object SparkSubmit { args.childArgs = ArrayBuffer(args.primaryResource, args.pyFiles) ++ args.childArgs args.files = mergeFileLists(args.files, args.primaryResource) } - val pyFiles = Option(args.pyFiles).getOrElse("") - args.files = mergeFileLists(args.files, pyFiles) - sysProps("spark.submit.pyFiles") = pyFiles + args.files = mergeFileLists(args.files, args.pyFiles) + // Format python file paths properly before adding them to the PYTHONPATH + sysProps("spark.submit.pyFiles") = PythonRunner.formatPaths(args.pyFiles).mkString(",") } // If we're deploying into YARN, use yarn.Client as a wrapper around the user class @@ -299,13 +299,18 @@ object SparkSubmit { } private def addJarToClasspath(localJar: String, loader: ExecutorURLClassLoader) { - val localJarFile = new File(localJar) - if (!localJarFile.exists()) { - printWarning(s"Jar $localJar does not exist, skipping.") + val uri = Utils.resolveURI(localJar) + uri.getScheme match { + case "file" | "local" => + val file = new File(uri.getPath) + if (file.exists()) { + loader.addURL(file.toURI.toURL) + } else { + printWarning(s"Local jar $file does not exist, skipping.") + } + case _ => + printWarning(s"Skip remote jar $uri.") } - - val url = localJarFile.getAbsoluteFile.toURI.toURL - loader.addURL(url) } /** @@ -318,7 +323,7 @@ object SparkSubmit { /** * Return whether the given primary resource represents a shell. */ - private def isShell(primaryResource: String): Boolean = { + private[spark] def isShell(primaryResource: String): Boolean = { primaryResource == SPARK_SHELL || primaryResource == PYSPARK_SHELL } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 0cc05fb95aef0..bf449afae695f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -118,7 +118,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class") } catch { case e: Exception => - SparkSubmit.printErrorAndExit("Failed to read JAR: " + primaryResource) + SparkSubmit.printErrorAndExit("Cannot load main class from JAR: " + primaryResource) return } } @@ -148,6 +148,18 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { SparkSubmit.printErrorAndExit("--py-files given but primary resource is not a Python script") } + // Require all python files to be local, so we can add them to the PYTHONPATH + if (isPython) { + if (Utils.nonLocalPaths(primaryResource).nonEmpty) { + SparkSubmit.printErrorAndExit(s"Only local python files are supported: $primaryResource") + } + val nonLocalPyFiles = Utils.nonLocalPaths(pyFiles).mkString(",") + if (nonLocalPyFiles.nonEmpty) { + SparkSubmit.printErrorAndExit( + s"Only local additional python files are supported: $nonLocalPyFiles") + } + } + if (master.startsWith("yarn")) { val hasHadoopEnv = sys.env.contains("HADOOP_CONF_DIR") || sys.env.contains("YARN_CONF_DIR") if (!hasHadoopEnv && !Utils.isTesting) { @@ -263,19 +275,19 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { parse(tail) case ("--files") :: value :: tail => - files = value + files = Utils.resolveURIs(value) parse(tail) case ("--py-files") :: value :: tail => - pyFiles = value + pyFiles = Utils.resolveURIs(value) parse(tail) case ("--archives") :: value :: tail => - archives = value + archives = Utils.resolveURIs(value) parse(tail) case ("--jars") :: value :: tail => - jars = value + jars = Utils.resolveURIs(value) parse(tail) case ("--help" | "-h") :: tail => @@ -296,7 +308,12 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { val errMessage = s"Unrecognized option '$value'." SparkSubmit.printErrorAndExit(errMessage) case v => - primaryResource = v + primaryResource = + if (!SparkSubmit.isShell(v)) { + Utils.resolveURI(v).toString + } else { + v + } inSparkOpts = false isPython = SparkSubmit.isPython(v) parse(tail) @@ -327,8 +344,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | --name NAME A name of your application. | --jars JARS Comma-separated list of local jars to include on the driver | and executor classpaths. - | --py-files PY_FILES Comma-separated list of .zip or .egg files to place on the - | PYTHONPATH for Python apps. + | --py-files PY_FILES Comma-separated list of .zip, .egg, or .py files to place + | on the PYTHONPATH for Python apps. | --files FILES Comma-separated list of files to be placed in the working | directory of each executor. | --properties-file FILE Path to a file from which to load extra properties. If not diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 0c7cff019fce1..3b1b6df089b8e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1086,9 +1086,19 @@ private[spark] object Utils extends Logging { } /** - * Return true if this is Windows. + * Whether the underlying operating system is Windows. */ - def isWindows = SystemUtils.IS_OS_WINDOWS + val isWindows = SystemUtils.IS_OS_WINDOWS + + /** + * Pattern for matching a Windows drive, which contains only a single alphabet character. + */ + val windowsDrive = "([a-zA-Z])".r + + /** + * Format a Windows path such that it can be safely passed to a URI. + */ + def formatWindowsPath(path: String): String = path.replace("\\", "/") /** * Indicates whether Spark is currently running unit tests. @@ -1166,4 +1176,61 @@ private[spark] object Utils extends Logging { true } } + + /** + * Return a well-formed URI for the file described by a user input string. + * + * If the supplied path does not contain a scheme, or is a relative path, it will be + * converted into an absolute path with a file:// scheme. + */ + def resolveURI(path: String, testWindows: Boolean = false): URI = { + + // In Windows, the file separator is a backslash, but this is inconsistent with the URI format + val windows = isWindows || testWindows + val formattedPath = if (windows) formatWindowsPath(path) else path + + val uri = new URI(formattedPath) + if (uri.getPath == null) { + throw new IllegalArgumentException(s"Given path is malformed: $uri") + } + uri.getScheme match { + case windowsDrive(d) if windows => + new URI("file:/" + uri.toString.stripPrefix("/")) + case null => + // Preserve fragments for HDFS file name substitution (denoted by "#") + // For instance, in "abc.py#xyz.py", "xyz.py" is the name observed by the application + val fragment = uri.getFragment + val part = new File(uri.getPath).toURI + new URI(part.getScheme, part.getPath, fragment) + case _ => + uri + } + } + + /** Resolve a comma-separated list of paths. */ + def resolveURIs(paths: String, testWindows: Boolean = false): String = { + if (paths == null || paths.trim.isEmpty) { + "" + } else { + paths.split(",").map { p => Utils.resolveURI(p, testWindows) }.mkString(",") + } + } + + /** Return all non-local paths from a comma-separated list of paths. */ + def nonLocalPaths(paths: String, testWindows: Boolean = false): Array[String] = { + val windows = isWindows || testWindows + if (paths == null || paths.trim.isEmpty) { + Array.empty + } else { + paths.split(",").filter { p => + val formattedPath = if (windows) formatWindowsPath(p) else p + new URI(formattedPath).getScheme match { + case windowsDrive(d) if windows => false + case "local" | "file" | null => false + case _ => true + } + } + } + } + } diff --git a/core/src/test/scala/org/apache/spark/deploy/PythonRunnerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/PythonRunnerSuite.scala new file mode 100644 index 0000000000000..bb6251fb4bfbe --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/PythonRunnerSuite.scala @@ -0,0 +1,61 @@ +/* + * 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.deploy + +import org.scalatest.FunSuite + +class PythonRunnerSuite extends FunSuite { + + // Test formatting a single path to be added to the PYTHONPATH + test("format path") { + assert(PythonRunner.formatPath("spark.py") === "spark.py") + assert(PythonRunner.formatPath("file:/spark.py") === "/spark.py") + assert(PythonRunner.formatPath("file:///spark.py") === "/spark.py") + assert(PythonRunner.formatPath("local:/spark.py") === "/spark.py") + assert(PythonRunner.formatPath("local:///spark.py") === "/spark.py") + assert(PythonRunner.formatPath("C:/a/b/spark.py", testWindows = true) === "C:/a/b/spark.py") + assert(PythonRunner.formatPath("/C:/a/b/spark.py", testWindows = true) === "C:/a/b/spark.py") + assert(PythonRunner.formatPath("file:/C:/a/b/spark.py", testWindows = true) === + "C:/a/b/spark.py") + intercept[IllegalArgumentException] { PythonRunner.formatPath("one:two") } + intercept[IllegalArgumentException] { PythonRunner.formatPath("hdfs:s3:xtremeFS") } + intercept[IllegalArgumentException] { PythonRunner.formatPath("hdfs:/path/to/some.py") } + } + + // Test formatting multiple comma-separated paths to be added to the PYTHONPATH + test("format paths") { + assert(PythonRunner.formatPaths("spark.py") === Array("spark.py")) + assert(PythonRunner.formatPaths("file:/spark.py") === Array("/spark.py")) + assert(PythonRunner.formatPaths("file:/app.py,local:/spark.py") === + Array("/app.py", "/spark.py")) + assert(PythonRunner.formatPaths("me.py,file:/you.py,local:/we.py") === + Array("me.py", "/you.py", "/we.py")) + assert(PythonRunner.formatPaths("C:/a/b/spark.py", testWindows = true) === + Array("C:/a/b/spark.py")) + assert(PythonRunner.formatPaths("/C:/a/b/spark.py", testWindows = true) === + Array("C:/a/b/spark.py")) + assert(PythonRunner.formatPaths("C:/free.py,pie.py", testWindows = true) === + Array("C:/free.py", "pie.py")) + assert(PythonRunner.formatPaths("lovely.py,C:/free.py,file:/d:/fry.py", testWindows = true) === + Array("lovely.py", "C:/free.py", "d:/fry.py")) + intercept[IllegalArgumentException] { PythonRunner.formatPaths("one:two,three") } + intercept[IllegalArgumentException] { PythonRunner.formatPaths("two,three,four:five:six") } + intercept[IllegalArgumentException] { PythonRunner.formatPaths("hdfs:/some.py,foo.py") } + intercept[IllegalArgumentException] { PythonRunner.formatPaths("foo.py,hdfs:/some.py") } + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 6c0deede53784..02427a4a83506 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -91,7 +91,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { "--jars=one.jar,two.jar,three.jar", "--name=myApp") val appArgs = new SparkSubmitArguments(clArgs) - appArgs.jars should be ("one.jar,two.jar,three.jar") + appArgs.jars should include regex (".*one.jar,.*two.jar,.*three.jar") appArgs.name should be ("myApp") } @@ -125,17 +125,17 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") - childArgsStr should include ("--jar thejar.jar") childArgsStr should include ("--class org.SomeClass") - childArgsStr should include ("--addJars one.jar,two.jar,three.jar") childArgsStr should include ("--executor-memory 5g") childArgsStr should include ("--driver-memory 4g") childArgsStr should include ("--executor-cores 5") childArgsStr should include ("--arg arg1 --arg arg2") childArgsStr should include ("--queue thequeue") - childArgsStr should include ("--files file1.txt,file2.txt") - childArgsStr should include ("--archives archive1.txt,archive2.txt") childArgsStr should include ("--num-executors 6") + childArgsStr should include regex ("--jar .*thejar.jar") + childArgsStr should include regex ("--addJars .*one.jar,.*two.jar,.*three.jar") + childArgsStr should include regex ("--files .*file1.txt,.*file2.txt") + childArgsStr should include regex ("--archives .*archive1.txt,.*archive2.txt") mainClass should be ("org.apache.spark.deploy.yarn.Client") classpath should have length (0) sysProps("spark.app.name") should be ("beauty") @@ -162,18 +162,19 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") mainClass should be ("org.SomeClass") - classpath should contain ("thejar.jar") - classpath should contain ("one.jar") - classpath should contain ("two.jar") - classpath should contain ("three.jar") + classpath should have length (4) + classpath(0) should endWith ("thejar.jar") + classpath(1) should endWith ("one.jar") + classpath(2) should endWith ("two.jar") + classpath(3) should endWith ("three.jar") sysProps("spark.app.name") should be ("trill") - sysProps("spark.jars") should be ("one.jar,two.jar,three.jar,thejar.jar") sysProps("spark.executor.memory") should be ("5g") sysProps("spark.executor.cores") should be ("5") sysProps("spark.yarn.queue") should be ("thequeue") - sysProps("spark.yarn.dist.files") should be ("file1.txt,file2.txt") - sysProps("spark.yarn.dist.archives") should be ("archive1.txt,archive2.txt") sysProps("spark.executor.instances") should be ("6") + sysProps("spark.yarn.dist.files") should include regex (".*file1.txt,.*file2.txt") + sysProps("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt") + sysProps("spark.jars") should include regex (".*one.jar,.*two.jar,.*three.jar,.*thejar.jar") sysProps("SPARK_SUBMIT") should be ("true") } @@ -190,11 +191,13 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") - childArgsStr.startsWith("--memory 4g --cores 5 --supervise") should be (true) - childArgsStr should include ("launch spark://h:p thejar.jar org.SomeClass arg1 arg2") + childArgsStr should startWith ("--memory 4g --cores 5 --supervise") + childArgsStr should include regex ("launch spark://h:p .*thejar.jar org.SomeClass arg1 arg2") mainClass should be ("org.apache.spark.deploy.Client") - classpath should have length (0) - sysProps should have size (2) // contains --jar entry and SPARK_SUBMIT + classpath should have size (0) + sysProps should have size (2) + sysProps.keys should contain ("spark.jars") + sysProps.keys should contain ("SPARK_SUBMIT") } test("handles standalone client mode") { @@ -211,7 +214,8 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") mainClass should be ("org.SomeClass") - classpath should contain ("thejar.jar") + classpath should have length (1) + classpath(0) should endWith ("thejar.jar") sysProps("spark.executor.memory") should be ("5g") sysProps("spark.cores.max") should be ("5") } @@ -230,7 +234,8 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") mainClass should be ("org.SomeClass") - classpath should contain ("thejar.jar") + classpath should have length (1) + classpath(0) should endWith ("thejar.jar") sysProps("spark.executor.memory") should be ("5g") sysProps("spark.cores.max") should be ("5") } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index cf9e20d347ddd..0aad882ed76a8 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.util import scala.util.Random import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} +import java.net.URI import java.nio.{ByteBuffer, ByteOrder} import com.google.common.base.Charsets @@ -168,5 +169,68 @@ class UtilsSuite extends FunSuite { assert(result.size.equals(1)) assert(result(0).getCanonicalPath.equals(child1.getCanonicalPath)) } -} + test("resolveURI") { + def assertResolves(before: String, after: String, testWindows: Boolean = false): Unit = { + assume(before.split(",").length == 1) + assert(Utils.resolveURI(before, testWindows) === new URI(after)) + assert(Utils.resolveURI(after, testWindows) === new URI(after)) + assert(new URI(Utils.resolveURIs(before, testWindows)) === new URI(after)) + assert(new URI(Utils.resolveURIs(after, testWindows)) === new URI(after)) + } + val cwd = System.getProperty("user.dir") + assertResolves("hdfs:/root/spark.jar", "hdfs:/root/spark.jar") + assertResolves("hdfs:///root/spark.jar#app.jar", "hdfs:/root/spark.jar#app.jar") + assertResolves("spark.jar", s"file:$cwd/spark.jar") + assertResolves("spark.jar#app.jar", s"file:$cwd/spark.jar#app.jar") + assertResolves("C:/path/to/file.txt", "file:/C:/path/to/file.txt", testWindows = true) + assertResolves("C:\\path\\to\\file.txt", "file:/C:/path/to/file.txt", testWindows = true) + assertResolves("file:/C:/path/to/file.txt", "file:/C:/path/to/file.txt", testWindows = true) + assertResolves("file:///C:/path/to/file.txt", "file:/C:/path/to/file.txt", testWindows = true) + assertResolves("file:/C:/file.txt#alias.txt", "file:/C:/file.txt#alias.txt", testWindows = true) + intercept[IllegalArgumentException] { Utils.resolveURI("file:foo") } + intercept[IllegalArgumentException] { Utils.resolveURI("file:foo:baby") } + + // Test resolving comma-delimited paths + assert(Utils.resolveURIs("jar1,jar2") === s"file:$cwd/jar1,file:$cwd/jar2") + assert(Utils.resolveURIs("file:/jar1,file:/jar2") === "file:/jar1,file:/jar2") + assert(Utils.resolveURIs("hdfs:/jar1,file:/jar2,jar3") === + s"hdfs:/jar1,file:/jar2,file:$cwd/jar3") + assert(Utils.resolveURIs("hdfs:/jar1,file:/jar2,jar3,jar4#jar5") === + s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:$cwd/jar4#jar5") + assert(Utils.resolveURIs("hdfs:/jar1,file:/jar2,jar3,C:\\pi.py#py.pi", testWindows = true) === + s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:/C:/pi.py#py.pi") + } + + test("nonLocalPaths") { + assert(Utils.nonLocalPaths("spark.jar") === Array.empty) + assert(Utils.nonLocalPaths("file:/spark.jar") === Array.empty) + assert(Utils.nonLocalPaths("file:///spark.jar") === Array.empty) + assert(Utils.nonLocalPaths("local:/spark.jar") === Array.empty) + assert(Utils.nonLocalPaths("local:///spark.jar") === Array.empty) + assert(Utils.nonLocalPaths("hdfs:/spark.jar") === Array("hdfs:/spark.jar")) + assert(Utils.nonLocalPaths("hdfs:///spark.jar") === Array("hdfs:///spark.jar")) + assert(Utils.nonLocalPaths("file:/spark.jar,local:/smart.jar,family.py") === Array.empty) + assert(Utils.nonLocalPaths("local:/spark.jar,file:/smart.jar,family.py") === Array.empty) + assert(Utils.nonLocalPaths("hdfs:/spark.jar,s3:/smart.jar") === + Array("hdfs:/spark.jar", "s3:/smart.jar")) + assert(Utils.nonLocalPaths("hdfs:/spark.jar,s3:/smart.jar,local.py,file:/hello/pi.py") === + Array("hdfs:/spark.jar", "s3:/smart.jar")) + assert(Utils.nonLocalPaths("local.py,hdfs:/spark.jar,file:/hello/pi.py,s3:/smart.jar") === + Array("hdfs:/spark.jar", "s3:/smart.jar")) + + // Test Windows paths + assert(Utils.nonLocalPaths("C:/some/path.jar", testWindows = true) === Array.empty) + assert(Utils.nonLocalPaths("file:/C:/some/path.jar", testWindows = true) === Array.empty) + assert(Utils.nonLocalPaths("file:///C:/some/path.jar", testWindows = true) === Array.empty) + assert(Utils.nonLocalPaths("local:/C:/some/path.jar", testWindows = true) === Array.empty) + assert(Utils.nonLocalPaths("local:///C:/some/path.jar", testWindows = true) === Array.empty) + assert(Utils.nonLocalPaths("hdfs:/a.jar,C:/my.jar,s3:/another.jar", testWindows = true) === + Array("hdfs:/a.jar", "s3:/another.jar")) + assert(Utils.nonLocalPaths("D:/your.jar,hdfs:/a.jar,s3:/another.jar", testWindows = true) === + Array("hdfs:/a.jar", "s3:/another.jar")) + assert(Utils.nonLocalPaths("hdfs:/a.jar,s3:/another.jar,e:/our.jar", testWindows = true) === + Array("hdfs:/a.jar", "s3:/another.jar")) + } + +} diff --git a/python/pyspark/context.py b/python/pyspark/context.py index c9ff82d23b3cf..27b440d73bdc3 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -159,10 +159,14 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, self.addPyFile(path) # Deploy code dependencies set by spark-submit; these will already have been added - # with SparkContext.addFile, so we just need to add them + # with SparkContext.addFile, so we just need to add them to the PYTHONPATH for path in self._conf.get("spark.submit.pyFiles", "").split(","): if path != "": - self._python_includes.append(os.path.basename(path)) + (dirname, filename) = os.path.split(path) + self._python_includes.append(filename) + sys.path.append(path) + if not dirname in sys.path: + sys.path.append(dirname) # Create a temporary directory inside spark.local.dir: local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir(self._jsc.sc().conf()) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 5f34362ccd973..e1db4d5395ab9 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -942,7 +942,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, def createSparkContext(): SparkContext = { val execUri = System.getenv("SPARK_EXECUTOR_URI") - val jars = SparkILoop.getAddedJars.map(new java.io.File(_).getAbsolutePath) + val jars = SparkILoop.getAddedJars val conf = new SparkConf() .setMaster(getMaster()) .setAppName("Spark shell") @@ -997,7 +997,8 @@ object SparkILoop { val propJars = sys.props.get("spark.jars").flatMap { p => if (p == "") None else Some(p) } - propJars.orElse(envJars).map(_.split(",")).getOrElse(Array.empty) + val jars = propJars.orElse(envJars).getOrElse("") + Utils.resolveURIs(jars).split(",").filter(_.nonEmpty) } // Designed primarily for use by test code: take a String with a From 75a03277704f8618a0f1c41aecfb1ebd24a8ac1a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 24 May 2014 18:27:00 -0700 Subject: [PATCH 414/641] SPARK-1911: Emphasize that Spark jars should be built with Java 6. This commit requires the user to manually say "yes" when buiding Spark without Java 6. The prompt can be bypassed with a flag (e.g. if the user is scripting around make-distribution). Author: Patrick Wendell Closes #859 from pwendell/java6 and squashes the following commits: 4921133 [Patrick Wendell] Adding Pyspark Notice fee8c9e [Patrick Wendell] SPARK-1911: Emphasize that Spark jars should be built with Java 6. --- make-distribution.sh | 52 ++++++++++++++++++++++++++------------------ 1 file changed, 31 insertions(+), 21 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index 1c89027d68bed..ae52b4976dc25 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -46,27 +46,6 @@ set -e FWDIR="$(cd `dirname $0`; pwd)" DISTDIR="$FWDIR/dist" -if [ -z "$JAVA_HOME" ]; then - echo "Error: JAVA_HOME is not set, cannot proceed." - exit -1 -fi - -JAVA_CMD="$JAVA_HOME"/bin/java -JAVA_VERSION=$("$JAVA_CMD" -version 2>&1) -if ! [[ "$JAVA_VERSION" =~ "1.6" ]]; then - echo "***NOTE***: JAVA_HOME is not set to a JDK 6 installation. The resulting" - echo " distribution will not support Java 6. See SPARK-1703." - echo "Output from 'java -version' was:" - echo "$JAVA_VERSION" -fi - -VERSION=$(mvn help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1) -if [ $? != 0 ]; then - echo -e "You need Maven installed to build Spark." - echo -e "Download Maven from https://maven.apache.org/" - exit -1; -fi - # Initialize defaults SPARK_HADOOP_VERSION=1.0.4 SPARK_YARN=false @@ -88,6 +67,9 @@ while (( "$#" )); do --with-hive) SPARK_HIVE=true ;; + --skip-java-test) + SKIP_JAVA_TEST=true + ;; --with-tachyon) SPARK_TACHYON=true ;; @@ -102,6 +84,34 @@ while (( "$#" )); do shift done +if [ -z "$JAVA_HOME" ]; then + echo "Error: JAVA_HOME is not set, cannot proceed." + exit -1 +fi + +VERSION=$(mvn help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1) +if [ $? != 0 ]; then + echo -e "You need Maven installed to build Spark." + echo -e "Download Maven from https://maven.apache.org/" + exit -1; +fi + +JAVA_CMD="$JAVA_HOME"/bin/java +JAVA_VERSION=$("$JAVA_CMD" -version 2>&1) +if [[ ! "$JAVA_VERSION" =~ "1.6" && -z "$SKIP_JAVA_TEST" ]]; then + echo "***NOTE***: JAVA_HOME is not set to a JDK 6 installation. The resulting" + echo " distribution may not work well with PySpark and will not run" + echo " with Java 6 (See SPARK-1703 and SPARK-1911)." + echo " This test can be disabled by adding --skip-java-test." + echo "Output from 'java -version' was:" + echo "$JAVA_VERSION" + read -p "Would you like to continue anyways? [y,n]: " -r + if [[ ! $REPLY =~ ^[Yy]$ ]]; then + echo "Okay, exiting." + exit 1 + fi +fi + if [ "$NAME" == "none" ]; then NAME=$SPARK_HADOOP_VERSION fi From 4e4831b8facc186cda6ef31040ccdeab48acbbb7 Mon Sep 17 00:00:00 2001 From: Zhen Peng Date: Sat, 24 May 2014 20:40:19 -0700 Subject: [PATCH 415/641] [SPARK-1886] check executor id existence when executor exit Author: Zhen Peng Closes #827 from zhpengg/bugfix-executor-id-not-found and squashes the following commits: cd8bb65 [Zhen Peng] bugfix: check executor id existence when executor exit --- .../apache/spark/deploy/worker/Worker.scala | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index fb9cc116cd08b..8b6747977eb87 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -263,14 +263,20 @@ private[spark] class Worker( } val fullId = appId + "/" + execId if (ExecutorState.isFinished(state)) { - val executor = executors(fullId) - logInfo("Executor " + fullId + " finished with state " + state + - message.map(" message " + _).getOrElse("") + - exitStatus.map(" exitStatus " + _).getOrElse("")) - executors -= fullId - finishedExecutors(fullId) = executor - coresUsed -= executor.cores - memoryUsed -= executor.memory + executors.get(fullId) match { + case Some(executor) => + logInfo("Executor " + fullId + " finished with state " + state + + message.map(" message " + _).getOrElse("") + + exitStatus.map(" exitStatus " + _).getOrElse("")) + executors -= fullId + finishedExecutors(fullId) = executor + coresUsed -= executor.cores + memoryUsed -= executor.memory + case None => + logInfo("Unknown Executor " + fullId + " finished with state " + state + + message.map(" message " + _).getOrElse("") + + exitStatus.map(" exitStatus " + _).getOrElse("")) + } } case KillExecutor(masterUrl, appId, execId) => From 5afe6af0b192ce7e908634992e8752537b1c4ed1 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 24 May 2014 20:42:01 -0700 Subject: [PATCH 416/641] [SPARK-1913][SQL] Bug fix: column pruning error in Parquet support JIRA issue: [SPARK-1913](https://issues.apache.org/jira/browse/SPARK-1913) When scanning Parquet tables, attributes referenced only in predicates that are pushed down are not passed to the `ParquetTableScan` operator and causes exception. Author: Cheng Lian Closes #863 from liancheng/spark-1913 and squashes the following commits: f976b73 [Cheng Lian] Addessed the readability issue commented by @rxin f5b257d [Cheng Lian] Added back comments deleted by mistake ae60ab3 [Cheng Lian] [SPARK-1913] Attributes referenced only in predicates pushed down should remain in ParquetTableScan operator --- .../org/apache/spark/sql/SQLContext.scala | 6 +++++- .../spark/sql/execution/SparkStrategies.scala | 20 ++++++++++--------- .../spark/sql/parquet/ParquetQuerySuite.scala | 6 +++++- .../spark/sql/hive/HiveStrategies.scala | 1 + 4 files changed, 22 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index bfebfa0c28c52..043be58edc91b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -206,17 +206,21 @@ class SQLContext(@transient val sparkContext: SparkContext) * final desired output requires complex expressions to be evaluated or when columns can be * further eliminated out after filtering has been done. * + * The `prunePushedDownFilters` parameter is used to remove those filters that can be optimized + * away by the filter pushdown optimization. + * * The required attributes for both filtering and expression evaluation are passed to the * provided `scanBuilder` function so that it can avoid unnecessary column materialization. */ def pruneFilterProject( projectList: Seq[NamedExpression], filterPredicates: Seq[Expression], + prunePushedDownFilters: Seq[Expression] => Seq[Expression], scanBuilder: Seq[Attribute] => SparkPlan): SparkPlan = { val projectSet = projectList.flatMap(_.references).toSet val filterSet = filterPredicates.flatMap(_.references).toSet - val filterCondition = filterPredicates.reduceLeftOption(And) + val filterCondition = prunePushedDownFilters(filterPredicates).reduceLeftOption(And) // Right now we still use a projection even if the only evaluation is applying an alias // to a column. Since this is a no-op, it could be avoided. However, using this diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 394a59700dbaf..cfa8bdae58b11 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -141,14 +141,14 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => InsertIntoParquetTable(table, planLater(child), overwrite)(sparkContext) :: Nil case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) => { - val remainingFilters = + val prunePushedDownFilters = if (sparkContext.conf.getBoolean(ParquetFilters.PARQUET_FILTER_PUSHDOWN_ENABLED, true)) { - filters.filter { - // Note: filters cannot be pushed down to Parquet if they contain more complex - // expressions than simple "Attribute cmp Literal" comparisons. Here we remove - // all filters that have been pushed down. Note that a predicate such as - // "(A AND B) OR C" can result in "A OR C" being pushed down. - filter => + (filters: Seq[Expression]) => { + filters.filter { filter => + // Note: filters cannot be pushed down to Parquet if they contain more complex + // expressions than simple "Attribute cmp Literal" comparisons. Here we remove + // all filters that have been pushed down. Note that a predicate such as + // "(A AND B) OR C" can result in "A OR C" being pushed down. val recordFilter = ParquetFilters.createFilter(filter) if (!recordFilter.isDefined) { // First case: the pushdown did not result in any record filter. @@ -159,13 +159,15 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // still want to keep "A AND B" in the higher-level filter, not just "B". !ParquetFilters.findExpression(recordFilter.get, filter).isDefined } + } } } else { - filters + identity[Seq[Expression]] _ } pruneFilterProject( projectList, - remainingFilters, + filters, + prunePushedDownFilters, ParquetTableScan(_, relation, filters)(sparkContext)) :: Nil } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 65f4c17aeee3a..f9731e82e4924 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -358,5 +358,9 @@ class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { assert(stringResult(0).getString(2) == "100", "stringvalue incorrect") assert(stringResult(0).getInt(1) === 100) } -} + test("SPARK-1913 regression: columns only referenced by pushed down filters should remain") { + val query = sql(s"SELECT mystring FROM testfiltersource WHERE myint < 10") + assert(query.collect().size === 10) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index b2157074a41bf..8b51957162e04 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -69,6 +69,7 @@ private[hive] trait HiveStrategies { pruneFilterProject( projectList, otherPredicates, + identity[Seq[Expression]], HiveTableScan(_, relation, pruningPredicates.reduceLeftOption(And))(hiveContext)) :: Nil case _ => Nil From 6e9fb6320bec3371bc9c010ccbc1b915f500486b Mon Sep 17 00:00:00 2001 From: Colin Patrick Mccabe Date: Sat, 24 May 2014 22:39:27 -0700 Subject: [PATCH 417/641] spark-submit: add exec at the end of the script Add an 'exec' at the end of the spark-submit script, to avoid keeping a bash process hanging around while it runs. This makes ps look a little bit nicer. Author: Colin Patrick Mccabe Closes #858 from cmccabe/SPARK-1907 and squashes the following commits: 7023b64 [Colin Patrick Mccabe] spark-submit: add exec at the end of the script --- bin/spark-submit | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/spark-submit b/bin/spark-submit index 63903b17a2902..9e7cecedd0325 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -41,5 +41,5 @@ if [ -n "$DRIVER_MEMORY" ] && [ $DEPLOY_MODE == "client" ]; then export SPARK_DRIVER_MEMORY=$DRIVER_MEMORY fi -$SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" +exec $SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" From 6052db9dc10c996215658485e805200e4f0cf549 Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Sun, 25 May 2014 00:06:42 -0700 Subject: [PATCH 418/641] [SPARK-1822] SchemaRDD.count() should use query optimizer Author: Kan Zhang Closes #841 from kanzhang/SPARK-1822 and squashes the following commits: 2f8072a [Kan Zhang] [SPARK-1822] Minor style update cf4baa4 [Kan Zhang] [SPARK-1822] Adding Scaladoc e67c910 [Kan Zhang] [SPARK-1822] SchemaRDD.count() should use optimizer --- python/pyspark/sql.py | 14 +++++++++++++- .../sql/catalyst/expressions/aggregates.scala | 6 +++--- .../scala/org/apache/spark/sql/SchemaRDD.scala | 9 +++++++++ .../scala/org/apache/spark/sql/DslQuerySuite.scala | 9 +++++---- .../test/scala/org/apache/spark/sql/TestData.scala | 2 ++ 5 files changed, 32 insertions(+), 8 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index bbe69e7d8f89b..f2001afae4ee5 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -268,7 +268,7 @@ def __init__(self, jschema_rdd, sql_ctx): def _jrdd(self): """ Lazy evaluation of PythonRDD object. Only done when a user calls methods defined by the - L{pyspark.rdd.RDD} super class (map, count, etc.). + L{pyspark.rdd.RDD} super class (map, filter, etc.). """ if not hasattr(self, '_lazy_jrdd'): self._lazy_jrdd = self._toPython()._jrdd @@ -321,6 +321,18 @@ def saveAsTable(self, tableName): """ self._jschema_rdd.saveAsTable(tableName) + def count(self): + """ + Return the number of elements in this RDD. + + >>> srdd = sqlCtx.inferSchema(rdd) + >>> srdd.count() + 3L + >>> srdd.count() == srdd.map(lambda x: x).count() + True + """ + return self._jschema_rdd.count() + def _toPython(self): # We have to import the Row class explicitly, so that the reference Pickler has is # pyspark.sql.Row instead of __main__.Row diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 5dbaaa3b0ce35..1bcd4e22766a9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -151,7 +151,7 @@ case class MaxFunction(expr: Expression, base: AggregateExpression) extends Aggr case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { override def references = child.references override def nullable = false - override def dataType = IntegerType + override def dataType = LongType override def toString = s"COUNT($child)" override def asPartial: SplitEvaluation = { @@ -295,12 +295,12 @@ case class AverageFunction(expr: Expression, base: AggregateExpression) case class CountFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. - var count: Int = _ + var count: Long = _ override def update(input: Row): Unit = { val evaluatedExpr = expr.map(_.eval(input)) if (evaluatedExpr.map(_ != null).reduceLeft(_ || _)) { - count += 1 + count += 1L } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 2569815ebb209..452da3d02310d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -274,6 +274,15 @@ class SchemaRDD( seed: Long) = new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan)) + /** + * :: Experimental :: + * Overriding base RDD implementation to leverage query optimizer + */ + @Experimental + override def count(): Long = { + groupBy()(Count(Literal(1))).collect().head.getLong(0) + } + /** * :: Experimental :: * Applies the given Generator, or table generating function, to this relation. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index f43e98d614094..233132a2fec69 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -108,10 +108,7 @@ class DslQuerySuite extends QueryTest { } test("count") { - checkAnswer( - testData2.groupBy()(Count(1)), - testData2.count() - ) + assert(testData2.count() === testData2.map(_ => 1).count()) } test("null count") { @@ -126,6 +123,10 @@ class DslQuerySuite extends QueryTest { ) } + test("zero count") { + assert(testData4.count() === 0) + } + test("inner join where, one match per row") { checkAnswer( upperCaseData.join(lowerCaseData, Inner).where('n === 'N), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 1aca3872524d7..b1eecb4dd3be4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -47,6 +47,8 @@ object TestData { (1, null) :: (2, 2) :: Nil) + val testData4 = logical.LocalRelation('a.int, 'b.int) + case class UpperCaseData(N: Int, L: String) val upperCaseData = TestSQLContext.sparkContext.parallelize( From 5c7faecd75ea59454ad3209390ac078e6cf6e4a6 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 25 May 2014 01:32:15 -0700 Subject: [PATCH 419/641] Added PEP8 style configuration file. This sets the max line length to 100 as a PEP8 exception. Author: Reynold Xin Closes #872 from rxin/pep8 and squashes the following commits: 2f26029 [Reynold Xin] Added PEP8 style configuration file. --- tox.ini | 2 ++ 1 file changed, 2 insertions(+) create mode 100644 tox.ini diff --git a/tox.ini b/tox.ini new file mode 100644 index 0000000000000..2d94b09a90a46 --- /dev/null +++ b/tox.ini @@ -0,0 +1,2 @@ +[pep8] +max-line-length=100 From d66642e3978a76977414c2fdaedebaad35662667 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 25 May 2014 01:44:49 -0700 Subject: [PATCH 420/641] SPARK-1822: Some minor cleanup work on SchemaRDD.count() Minor cleanup following #841. Author: Reynold Xin Closes #868 from rxin/schema-count and squashes the following commits: 5442651 [Reynold Xin] SPARK-1822: Some minor cleanup work on SchemaRDD.count() --- python/pyspark/sql.py | 5 ++++- .../src/main/scala/org/apache/spark/sql/SchemaRDD.scala | 8 ++++---- .../test/scala/org/apache/spark/sql/DslQuerySuite.scala | 2 +- .../src/test/scala/org/apache/spark/sql/TestData.scala | 2 +- 4 files changed, 10 insertions(+), 7 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index f2001afae4ee5..fa4b9c7b688ea 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -323,7 +323,10 @@ def saveAsTable(self, tableName): def count(self): """ - Return the number of elements in this RDD. + Return the number of elements in this RDD. Unlike the base RDD + implementation of count, this implementation leverages the query + optimizer to compute the count on the SchemaRDD, which supports + features such as filter pushdown. >>> srdd = sqlCtx.inferSchema(rdd) >>> srdd.count() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 452da3d02310d..9883ebc0b3c62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -276,12 +276,12 @@ class SchemaRDD( /** * :: Experimental :: - * Overriding base RDD implementation to leverage query optimizer + * Return the number of elements in the RDD. Unlike the base RDD implementation of count, this + * implementation leverages the query optimizer to compute the count on the SchemaRDD, which + * supports features such as filter pushdown. */ @Experimental - override def count(): Long = { - groupBy()(Count(Literal(1))).collect().head.getLong(0) - } + override def count(): Long = groupBy()(Count(Literal(1))).collect().head.getLong(0) /** * :: Experimental :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 233132a2fec69..94ba13b14b33d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -124,7 +124,7 @@ class DslQuerySuite extends QueryTest { } test("zero count") { - assert(testData4.count() === 0) + assert(emptyTableData.count() === 0) } test("inner join where, one match per row") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index b1eecb4dd3be4..944f520e43515 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -47,7 +47,7 @@ object TestData { (1, null) :: (2, 2) :: Nil) - val testData4 = logical.LocalRelation('a.int, 'b.int) + val emptyTableData = logical.LocalRelation('a.int, 'b.int) case class UpperCaseData(N: Int, L: String) val upperCaseData = From 55fddf9cc0fe420d5396b0e730c8413b2f23d636 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 25 May 2014 01:47:08 -0700 Subject: [PATCH 421/641] Added license header for tox.ini. (cherry picked from commit fa541f32c5b92e6868a9c99cbb2c87115d624d23) Signed-off-by: Reynold Xin --- tox.ini | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/tox.ini b/tox.ini index 2d94b09a90a46..44766e529bf7f 100644 --- a/tox.ini +++ b/tox.ini @@ -1,2 +1,17 @@ +# 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. + [pep8] max-line-length=100 From d79c2b28e17ec0b15198aaedd2e1f403d81f717e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 25 May 2014 14:48:27 -0700 Subject: [PATCH 422/641] Fix PEP8 violations in examples/src/main/python. Author: Reynold Xin Closes #870 from rxin/examples-python-pep8 and squashes the following commits: 2829e84 [Reynold Xin] Fix PEP8 violations in examples/src/main/python. --- examples/src/main/python/als.py | 20 +++++++++++-------- examples/src/main/python/kmeans.py | 2 +- .../src/main/python/logistic_regression.py | 4 ++-- examples/src/main/python/pagerank.py | 12 +++++------ examples/src/main/python/pi.py | 2 ++ examples/src/main/python/sort.py | 4 ++-- 6 files changed, 25 insertions(+), 19 deletions(-) diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index f0b46cd28b7aa..1a7c4c51f48cd 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -29,22 +29,25 @@ LAMBDA = 0.01 # regularization np.random.seed(42) + def rmse(R, ms, us): diff = R - ms * us.T return np.sqrt(np.sum(np.power(diff, 2)) / M * U) + def update(i, vec, mat, ratings): uu = mat.shape[0] ff = mat.shape[1] - + XtX = mat.T * mat Xty = mat.T * ratings[i, :].T - + for j in range(ff): - XtX[j,j] += LAMBDA * uu - + XtX[j, j] += LAMBDA * uu + return np.linalg.solve(XtX, Xty) + if __name__ == "__main__": """ Usage: als [M] [U] [F] [iterations] [slices]" @@ -57,10 +60,10 @@ def update(i, vec, mat, ratings): slices = int(sys.argv[5]) if len(sys.argv) > 5 else 2 print "Running ALS with M=%d, U=%d, F=%d, iters=%d, slices=%d\n" % \ - (M, U, F, ITERATIONS, slices) + (M, U, F, ITERATIONS, slices) R = matrix(rand(M, F)) * matrix(rand(U, F).T) - ms = matrix(rand(M ,F)) + ms = matrix(rand(M, F)) us = matrix(rand(U, F)) Rb = sc.broadcast(R) @@ -71,8 +74,9 @@ def update(i, vec, mat, ratings): ms = sc.parallelize(range(M), slices) \ .map(lambda x: update(x, msb.value[x, :], usb.value, Rb.value)) \ .collect() - ms = matrix(np.array(ms)[:, :, 0]) # collect() returns a list, so array ends up being - # a 3-d array, we take the first 2 dims for the matrix + # collect() returns a list, so array ends up being + # a 3-d array, we take the first 2 dims for the matrix + ms = matrix(np.array(ms)[:, :, 0]) msb = sc.broadcast(ms) us = sc.parallelize(range(U), slices) \ diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index fc16586c28a46..988fc45baf3bc 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -59,7 +59,7 @@ def closestPoint(p, centers): while tempDist > convergeDist: closest = data.map( - lambda p : (closestPoint(p, kPoints), (p, 1))) + lambda p: (closestPoint(p, kPoints), (p, 1))) pointStats = closest.reduceByKey( lambda (x1, y1), (x2, y2): (x1 + x2, y1 + y2)) newPoints = pointStats.map( diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index 0f22d0b32319e..6c33deabfd6ea 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -60,8 +60,8 @@ def readPointBatch(iterator): # Compute logistic regression gradient for a matrix of data points def gradient(matrix, w): - Y = matrix[:,0] # point labels (first column of input file) - X = matrix[:,1:] # point coordinates + Y = matrix[:, 0] # point labels (first column of input file) + X = matrix[:, 1:] # point coordinates # For each point (x, y), compute gradient function, then sum these up return ((1.0 / (1.0 + np.exp(-Y * X.dot(w))) - 1.0) * Y * X.T).sum(1) diff --git a/examples/src/main/python/pagerank.py b/examples/src/main/python/pagerank.py index d350fa46fa49a..0b96343158d44 100755 --- a/examples/src/main/python/pagerank.py +++ b/examples/src/main/python/pagerank.py @@ -15,9 +15,8 @@ # limitations under the License. # -#!/usr/bin/env python - -import re, sys +import re +import sys from operator import add from pyspark import SparkContext @@ -26,7 +25,8 @@ def computeContribs(urls, rank): """Calculates URL contributions to the rank of other URLs.""" num_urls = len(urls) - for url in urls: yield (url, rank / num_urls) + for url in urls: + yield (url, rank / num_urls) def parseNeighbors(urls): @@ -59,8 +59,8 @@ def parseNeighbors(urls): # Calculates and updates URL ranks continuously using PageRank algorithm. for iteration in xrange(int(sys.argv[2])): # Calculates URL contributions to the rank of other URLs. - contribs = links.join(ranks).flatMap(lambda (url, (urls, rank)): - computeContribs(urls, rank)) + contribs = links.join(ranks).flatMap( + lambda (url, (urls, rank)): computeContribs(urls, rank)) # Re-calculates URL ranks based on neighbor contributions. ranks = contribs.reduceByKey(add).mapValues(lambda rank: rank * 0.85 + 0.15) diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index 234720b55fa49..21d94a2cd4b64 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -29,9 +29,11 @@ sc = SparkContext(appName="PythonPi") slices = int(sys.argv[1]) if len(sys.argv) > 1 else 2 n = 100000 * slices + def f(_): x = random() * 2 - 1 y = random() * 2 - 1 return 1 if x ** 2 + y ** 2 < 1 else 0 + count = sc.parallelize(xrange(1, n+1), slices).map(f).reduce(add) print "Pi is roughly %f" % (4.0 * count / n) diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py index 4913ee926aa03..41d00c1b79133 100755 --- a/examples/src/main/python/sort.py +++ b/examples/src/main/python/sort.py @@ -27,8 +27,8 @@ sc = SparkContext(appName="PythonSort") lines = sc.textFile(sys.argv[1], 1) sortedCount = lines.flatMap(lambda x: x.split(' ')) \ - .map(lambda x: (int(x), 1)) \ - .sortByKey(lambda x: x) + .map(lambda x: (int(x), 1)) \ + .sortByKey(lambda x: x) # This is just a demo on how to bring all the sorted data back to a single node. # In reality, we wouldn't want to collect all the data to the driver node. output = sortedCount.collect() From 14f0358b2a0a9b92526bdad6d501ab753459eaa0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 25 May 2014 16:04:17 -0700 Subject: [PATCH 423/641] Python docstring update for sql.py. Mostly related to the following two rules in PEP8 and PEP257: - Line length < 72 chars. - First line should be a concise description of the function/class. Author: Reynold Xin Closes #869 from rxin/docstring-schemardd and squashes the following commits: 7cf0cbc [Reynold Xin] Updated sql.py for pep8 docstring. 0a4aef9 [Reynold Xin] Merge branch 'master' into docstring-schemardd 6678937 [Reynold Xin] Python docstring update for sql.py. --- python/pyspark/sql.py | 124 +++++++++++++++++++++--------------------- 1 file changed, 63 insertions(+), 61 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index fa4b9c7b688ea..b4e9618cc25b5 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -23,14 +23,14 @@ class SQLContext: - """ - Main entry point for SparkSQL functionality. A SQLContext can be used create L{SchemaRDD}s, - register L{SchemaRDD}s as tables, execute sql over tables, cache tables, and read parquet files. + """Main entry point for SparkSQL functionality. + + A SQLContext can be used create L{SchemaRDD}s, register L{SchemaRDD}s as + tables, execute SQL over tables, cache tables, and read parquet files. """ def __init__(self, sparkContext, sqlContext = None): - """ - Create a new SQLContext. + """Create a new SQLContext. @param sparkContext: The SparkContext to wrap. @@ -63,18 +63,20 @@ def __init__(self, sparkContext, sqlContext = None): @property def _ssql_ctx(self): - """ - Accessor for the JVM SparkSQL context. Subclasses can override this property to provide - their own JVM Contexts. + """Accessor for the JVM SparkSQL context. + + Subclasses can override this property to provide their own + JVM Contexts. """ if not hasattr(self, '_scala_SQLContext'): self._scala_SQLContext = self._jvm.SQLContext(self._jsc.sc()) return self._scala_SQLContext def inferSchema(self, rdd): - """ - Infer and apply a schema to an RDD of L{dict}s. We peek at the first row of the RDD to - determine the fields names and types, and then use that to extract all the dictionaries. + """Infer and apply a schema to an RDD of L{dict}s. + + We peek at the first row of the RDD to determine the fields names + and types, and then use that to extract all the dictionaries. >>> srdd = sqlCtx.inferSchema(rdd) >>> srdd.collect() == [{"field1" : 1, "field2" : "row1"}, {"field1" : 2, "field2": "row2"}, @@ -92,9 +94,10 @@ def inferSchema(self, rdd): return SchemaRDD(srdd, self) def registerRDDAsTable(self, rdd, tableName): - """ - Registers the given RDD as a temporary table in the catalog. Temporary tables exist only - during the lifetime of this instance of SQLContext. + """Registers the given RDD as a temporary table in the catalog. + + Temporary tables exist only during the lifetime of this instance of + SQLContext. >>> srdd = sqlCtx.inferSchema(rdd) >>> sqlCtx.registerRDDAsTable(srdd, "table1") @@ -106,8 +109,7 @@ def registerRDDAsTable(self, rdd, tableName): raise ValueError("Can only register SchemaRDD as table") def parquetFile(self, path): - """ - Loads a Parquet file, returning the result as a L{SchemaRDD}. + """Loads a Parquet file, returning the result as a L{SchemaRDD}. >>> import tempfile, shutil >>> parquetFile = tempfile.mkdtemp() @@ -122,8 +124,7 @@ def parquetFile(self, path): return SchemaRDD(jschema_rdd, self) def sql(self, sqlQuery): - """ - Executes a SQL query using Spark, returning the result as a L{SchemaRDD}. + """Return a L{SchemaRDD} representing the result of the given query. >>> srdd = sqlCtx.inferSchema(rdd) >>> sqlCtx.registerRDDAsTable(srdd, "table1") @@ -135,8 +136,7 @@ def sql(self, sqlQuery): return SchemaRDD(self._ssql_ctx.sql(sqlQuery), self) def table(self, tableName): - """ - Returns the specified table as a L{SchemaRDD}. + """Returns the specified table as a L{SchemaRDD}. >>> srdd = sqlCtx.inferSchema(rdd) >>> sqlCtx.registerRDDAsTable(srdd, "table1") @@ -147,23 +147,19 @@ def table(self, tableName): return SchemaRDD(self._ssql_ctx.table(tableName), self) def cacheTable(self, tableName): - """ - Caches the specified table in-memory. - """ + """Caches the specified table in-memory.""" self._ssql_ctx.cacheTable(tableName) def uncacheTable(self, tableName): - """ - Removes the specified table from the in-memory cache. - """ + """Removes the specified table from the in-memory cache.""" self._ssql_ctx.uncacheTable(tableName) class HiveContext(SQLContext): - """ - An instance of the Spark SQL execution engine that integrates with data stored in Hive. - Configuration for Hive is read from hive-site.xml on the classpath. It supports running both SQL - and HiveQL commands. + """A variant of Spark SQL that integrates with data stored in Hive. + + Configuration for Hive is read from hive-site.xml on the classpath. + It supports running both SQL and HiveQL commands. """ @property @@ -193,9 +189,10 @@ def hql(self, hqlQuery): class LocalHiveContext(HiveContext): - """ - Starts up an instance of hive where metadata is stored locally. An in-process metadata data is - created with data stored in ./metadata. Warehouse data is stored in in ./warehouse. + """Starts up an instance of hive where metadata is stored locally. + + An in-process metadata data is created with data stored in ./metadata. + Warehouse data is stored in in ./warehouse. >>> import os >>> hiveCtx = LocalHiveContext(sc) @@ -228,8 +225,10 @@ def _get_hive_ctx(self): # TODO: Investigate if it is more efficient to use a namedtuple. One problem is that named tuples # are custom classes that must be generated per Schema. class Row(dict): - """ - An extended L{dict} that takes a L{dict} in its constructor, and exposes those items as fields. + """A row in L{SchemaRDD}. + + An extended L{dict} that takes a L{dict} in its constructor, and + exposes those items as fields. >>> r = Row({"hello" : "world", "foo" : "bar"}) >>> r.hello @@ -245,13 +244,16 @@ def __init__(self, d): class SchemaRDD(RDD): - """ - An RDD of L{Row} objects that has an associated schema. The underlying JVM object is a SchemaRDD, - not a PythonRDD, so we can utilize the relational query api exposed by SparkSQL. + """An RDD of L{Row} objects that has an associated schema. - For normal L{pyspark.rdd.RDD} operations (map, count, etc.) the L{SchemaRDD} is not operated on - directly, as it's underlying implementation is a RDD composed of Java objects. Instead it is - converted to a PythonRDD in the JVM, on which Python operations can be done. + The underlying JVM object is a SchemaRDD, not a PythonRDD, so we can + utilize the relational query api exposed by SparkSQL. + + For normal L{pyspark.rdd.RDD} operations (map, count, etc.) the + L{SchemaRDD} is not operated on directly, as it's underlying + implementation is a RDD composed of Java objects. Instead it is + converted to a PythonRDD in the JVM, on which Python operations can + be done. """ def __init__(self, jschema_rdd, sql_ctx): @@ -266,8 +268,9 @@ def __init__(self, jschema_rdd, sql_ctx): @property def _jrdd(self): - """ - Lazy evaluation of PythonRDD object. Only done when a user calls methods defined by the + """Lazy evaluation of PythonRDD object. + + Only done when a user calls methods defined by the L{pyspark.rdd.RDD} super class (map, filter, etc.). """ if not hasattr(self, '_lazy_jrdd'): @@ -279,10 +282,10 @@ def _id(self): return self._jrdd.id() def saveAsParquetFile(self, path): - """ - Saves the contents of this L{SchemaRDD} as a parquet file, preserving the schema. Files - that are written out using this method can be read back in as a SchemaRDD using the - L{SQLContext.parquetFile} method. + """Save the contents as a Parquet file, preserving the schema. + + Files that are written out using this method can be read back in as + a SchemaRDD using the L{SQLContext.parquetFile} method. >>> import tempfile, shutil >>> parquetFile = tempfile.mkdtemp() @@ -296,9 +299,10 @@ def saveAsParquetFile(self, path): self._jschema_rdd.saveAsParquetFile(path) def registerAsTable(self, name): - """ - Registers this RDD as a temporary table using the given name. The lifetime of this temporary - table is tied to the L{SQLContext} that was used to create this SchemaRDD. + """Registers this RDD as a temporary table using the given name. + + The lifetime of this temporary table is tied to the L{SQLContext} + that was used to create this SchemaRDD. >>> srdd = sqlCtx.inferSchema(rdd) >>> srdd.registerAsTable("test") @@ -309,24 +313,22 @@ def registerAsTable(self, name): self._jschema_rdd.registerAsTable(name) def insertInto(self, tableName, overwrite = False): - """ - Inserts the contents of this SchemaRDD into the specified table, - optionally overwriting any existing data. + """Inserts the contents of this SchemaRDD into the specified table. + + Optionally overwriting any existing data. """ self._jschema_rdd.insertInto(tableName, overwrite) def saveAsTable(self, tableName): - """ - Creates a new table with the contents of this SchemaRDD. - """ + """Creates a new table with the contents of this SchemaRDD.""" self._jschema_rdd.saveAsTable(tableName) def count(self): - """ - Return the number of elements in this RDD. Unlike the base RDD - implementation of count, this implementation leverages the query - optimizer to compute the count on the SchemaRDD, which supports - features such as filter pushdown. + """Return the number of elements in this RDD. + + Unlike the base RDD implementation of count, this implementation + leverages the query optimizer to compute the count on the SchemaRDD, + which supports features such as filter pushdown. >>> srdd = sqlCtx.inferSchema(rdd) >>> srdd.count() From d33d3c61ae9e4551aed0217e525a109e678298f2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 25 May 2014 17:15:01 -0700 Subject: [PATCH 424/641] Fix PEP8 violations in Python mllib. Author: Reynold Xin Closes #871 from rxin/mllib-pep8 and squashes the following commits: 848416f [Reynold Xin] Fixed a typo in the previous cleanup (c -> sc). a8db4cd [Reynold Xin] Fix PEP8 violations in Python mllib. --- python/pyspark/mllib/_common.py | 42 +++++++++++++------------- python/pyspark/mllib/classification.py | 26 ++++++++-------- python/pyspark/mllib/clustering.py | 15 +++++---- python/pyspark/mllib/linalg.py | 13 ++++---- python/pyspark/mllib/recommendation.py | 15 +++++---- python/pyspark/mllib/regression.py | 24 ++++++--------- python/pyspark/mllib/tests.py | 27 ++++++----------- python/pyspark/mllib/util.py | 4 +-- 8 files changed, 78 insertions(+), 88 deletions(-) diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index e6f0953810ed7..802a27a8da14d 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -56,7 +56,8 @@ # # Sparse double vector format: # -# [1-byte 2] [4-byte length] [4-byte nonzeros] [nonzeros*4 bytes of indices] [nonzeros*8 bytes of values] +# [1-byte 2] [4-byte length] [4-byte nonzeros] [nonzeros*4 bytes of indices] \ +# [nonzeros*8 bytes of values] # # Double matrix format: # @@ -110,18 +111,18 @@ def _serialize_double_vector(v): return _serialize_sparse_vector(v) else: raise TypeError("_serialize_double_vector called on a %s; " - "wanted ndarray or SparseVector" % type(v)) + "wanted ndarray or SparseVector" % type(v)) def _serialize_dense_vector(v): """Serialize a dense vector given as a NumPy array.""" if v.ndim != 1: raise TypeError("_serialize_double_vector called on a %ddarray; " - "wanted a 1darray" % v.ndim) + "wanted a 1darray" % v.ndim) if v.dtype != float64: if numpy.issubdtype(v.dtype, numpy.complex): raise TypeError("_serialize_double_vector called on an ndarray of %s; " - "wanted ndarray of float64" % v.dtype) + "wanted ndarray of float64" % v.dtype) v = v.astype(float64) length = v.shape[0] ba = bytearray(5 + 8 * length) @@ -158,10 +159,10 @@ def _deserialize_double_vector(ba): """ if type(ba) != bytearray: raise TypeError("_deserialize_double_vector called on a %s; " - "wanted bytearray" % type(ba)) + "wanted bytearray" % type(ba)) if len(ba) < 5: raise TypeError("_deserialize_double_vector called on a %d-byte array, " - "which is too short" % len(ba)) + "which is too short" % len(ba)) if ba[0] == DENSE_VECTOR_MAGIC: return _deserialize_dense_vector(ba) elif ba[0] == SPARSE_VECTOR_MAGIC: @@ -175,7 +176,7 @@ def _deserialize_dense_vector(ba): """Deserialize a dense vector into a numpy array.""" if len(ba) < 5: raise TypeError("_deserialize_dense_vector called on a %d-byte array, " - "which is too short" % len(ba)) + "which is too short" % len(ba)) length = ndarray(shape=[1], buffer=ba, offset=1, dtype=int32)[0] if len(ba) != 8 * length + 5: raise TypeError("_deserialize_dense_vector called on bytearray " @@ -187,7 +188,7 @@ def _deserialize_sparse_vector(ba): """Deserialize a sparse vector into a MLlib SparseVector object.""" if len(ba) < 9: raise TypeError("_deserialize_sparse_vector called on a %d-byte array, " - "which is too short" % len(ba)) + "which is too short" % len(ba)) header = ndarray(shape=[2], buffer=ba, offset=1, dtype=int32) size = header[0] nonzeros = header[1] @@ -205,7 +206,7 @@ def _serialize_double_matrix(m): if m.dtype != float64: if numpy.issubdtype(m.dtype, numpy.complex): raise TypeError("_serialize_double_matrix called on an ndarray of %s; " - "wanted ndarray of float64" % m.dtype) + "wanted ndarray of float64" % m.dtype) m = m.astype(float64) rows = m.shape[0] cols = m.shape[1] @@ -225,10 +226,10 @@ def _deserialize_double_matrix(ba): """Deserialize a double matrix from a mutually understood format.""" if type(ba) != bytearray: raise TypeError("_deserialize_double_matrix called on a %s; " - "wanted bytearray" % type(ba)) + "wanted bytearray" % type(ba)) if len(ba) < 9: raise TypeError("_deserialize_double_matrix called on a %d-byte array, " - "which is too short" % len(ba)) + "which is too short" % len(ba)) if ba[0] != DENSE_MATRIX_MAGIC: raise TypeError("_deserialize_double_matrix called on bytearray " "with wrong magic") @@ -267,7 +268,7 @@ def _copyto(array, buffer, offset, shape, dtype): def _get_unmangled_rdd(data, serializer): dataBytes = data.map(serializer) dataBytes._bypass_serializer = True - dataBytes.cache() # TODO: users should unpersist() this later! + dataBytes.cache() # TODO: users should unpersist() this later! return dataBytes @@ -293,14 +294,14 @@ def _linear_predictor_typecheck(x, coeffs): if type(x) == ndarray: if x.ndim == 1: if x.shape != coeffs.shape: - raise RuntimeError("Got array of %d elements; wanted %d" - % (numpy.shape(x)[0], coeffs.shape[0])) + raise RuntimeError("Got array of %d elements; wanted %d" % ( + numpy.shape(x)[0], coeffs.shape[0])) else: raise RuntimeError("Bulk predict not yet supported.") elif type(x) == SparseVector: if x.size != coeffs.shape[0]: - raise RuntimeError("Got sparse vector of size %d; wanted %d" - % (x.size, coeffs.shape[0])) + raise RuntimeError("Got sparse vector of size %d; wanted %d" % ( + x.size, coeffs.shape[0])) elif (type(x) == RDD): raise RuntimeError("Bulk predict not yet supported.") else: @@ -315,7 +316,7 @@ def _get_initial_weights(initial_weights, data): if type(initial_weights) == ndarray: if initial_weights.ndim != 1: raise TypeError("At least one data element has " - + initial_weights.ndim + " dimensions, which is not 1") + + initial_weights.ndim + " dimensions, which is not 1") initial_weights = numpy.zeros([initial_weights.shape[0]]) elif type(initial_weights) == SparseVector: initial_weights = numpy.zeros([initial_weights.size]) @@ -333,10 +334,10 @@ def _regression_train_wrapper(sc, train_func, klass, data, initial_weights): raise RuntimeError("JVM call result had unexpected length") elif type(ans[0]) != bytearray: raise RuntimeError("JVM call result had first element of type " - + type(ans[0]).__name__ + " which is not bytearray") + + type(ans[0]).__name__ + " which is not bytearray") elif type(ans[1]) != float: raise RuntimeError("JVM call result had second element of type " - + type(ans[0]).__name__ + " which is not float") + + type(ans[0]).__name__ + " which is not float") return klass(_deserialize_double_vector(ans[0]), ans[1]) @@ -450,8 +451,7 @@ def _test(): import doctest globs = globals().copy() globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) - (failure_count, test_count) = doctest.testmod(globs=globs, - optionflags=doctest.ELLIPSIS) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 6772e4337ef39..1c0c536c4fb3d 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -29,6 +29,7 @@ from pyspark.mllib.regression import LabeledPoint, LinearModel from math import exp, log + class LogisticRegressionModel(LinearModel): """A linear binary classification model derived from logistic regression. @@ -68,14 +69,14 @@ def predict(self, x): class LogisticRegressionWithSGD(object): @classmethod - def train(cls, data, iterations=100, step=1.0, - miniBatchFraction=1.0, initialWeights=None): + def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, initialWeights=None): """Train a logistic regression model on the given data.""" sc = data.context - return _regression_train_wrapper(sc, lambda d, i: - sc._jvm.PythonMLLibAPI().trainLogisticRegressionModelWithSGD(d._jrdd, - iterations, step, miniBatchFraction, i), - LogisticRegressionModel, data, initialWeights) + train_func = lambda d, i: sc._jvm.PythonMLLibAPI().trainLogisticRegressionModelWithSGD( + d._jrdd, iterations, step, miniBatchFraction, i) + return _regression_train_wrapper(sc, train_func, LogisticRegressionModel, data, + initialWeights) + class SVMModel(LinearModel): """A support vector machine. @@ -106,16 +107,17 @@ def predict(self, x): margin = _dot(x, self._coeff) + self._intercept return 1 if margin >= 0 else 0 + class SVMWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): """Train a support vector machine on the given data.""" sc = data.context - return _regression_train_wrapper(sc, lambda d, i: - sc._jvm.PythonMLLibAPI().trainSVMModelWithSGD(d._jrdd, - iterations, step, regParam, miniBatchFraction, i), - SVMModel, data, initialWeights) + train_func = lambda d, i: sc._jvm.PythonMLLibAPI().trainSVMModelWithSGD( + d._jrdd, iterations, step, regParam, miniBatchFraction, i) + return _regression_train_wrapper(sc, train_func, SVMModel, data, initialWeights) + class NaiveBayesModel(object): """ @@ -156,6 +158,7 @@ def predict(self, x): """Return the most likely class for a data vector x""" return self.labels[numpy.argmax(self.pi + _dot(x, self.theta.transpose()))] + class NaiveBayes(object): @classmethod def train(cls, data, lambda_=1.0): @@ -186,8 +189,7 @@ def _test(): import doctest globs = globals().copy() globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) - (failure_count, test_count) = doctest.testmod(globs=globs, - optionflags=doctest.ELLIPSIS) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index f65088c9170e0..b380e8f6c8725 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -30,7 +30,8 @@ class KMeansModel(object): """A clustering model derived from the k-means method. >>> data = array([0.0,0.0, 1.0,1.0, 9.0,8.0, 8.0,9.0]).reshape(4,2) - >>> model = KMeans.train(sc.parallelize(data), 2, maxIterations=10, runs=30, initializationMode="random") + >>> model = KMeans.train( + ... sc.parallelize(data), 2, maxIterations=10, runs=30, initializationMode="random") >>> model.predict(array([0.0, 0.0])) == model.predict(array([1.0, 1.0])) True >>> model.predict(array([8.0, 9.0])) == model.predict(array([9.0, 8.0])) @@ -76,18 +77,17 @@ def predict(self, x): class KMeans(object): @classmethod - def train(cls, data, k, maxIterations=100, runs=1, - initializationMode="k-means||"): + def train(cls, data, k, maxIterations=100, runs=1, initializationMode="k-means||"): """Train a k-means clustering model.""" sc = data.context dataBytes = _get_unmangled_double_vector_rdd(data) - ans = sc._jvm.PythonMLLibAPI().trainKMeansModel(dataBytes._jrdd, - k, maxIterations, runs, initializationMode) + ans = sc._jvm.PythonMLLibAPI().trainKMeansModel( + dataBytes._jrdd, k, maxIterations, runs, initializationMode) if len(ans) != 1: raise RuntimeError("JVM call result had unexpected length") elif type(ans[0]) != bytearray: raise RuntimeError("JVM call result had first element of type " - + type(ans[0]) + " which is not bytearray") + + type(ans[0]) + " which is not bytearray") matrix = _deserialize_double_matrix(ans[0]) return KMeansModel([row for row in matrix]) @@ -96,8 +96,7 @@ def _test(): import doctest globs = globals().copy() globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) - (failure_count, test_count) = doctest.testmod(globs=globs, - optionflags=doctest.ELLIPSIS) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 7511ca7573ddb..276684272068b 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -54,7 +54,7 @@ def __init__(self, size, *args): if len(args) == 1: pairs = args[0] if type(pairs) == dict: - pairs = pairs.items() + pairs = pairs.items() pairs = sorted(pairs) self.indices = array([p[0] for p in pairs], dtype=int32) self.values = array([p[1] for p in pairs], dtype=float64) @@ -88,7 +88,7 @@ def dot(self, other): result += self.values[i] * other[self.indices[i]] return result elif other.ndim == 2: - results = [self.dot(other[:,i]) for i in xrange(other.shape[1])] + results = [self.dot(other[:, i]) for i in xrange(other.shape[1])] return array(results) else: raise Exception("Cannot call dot with %d-dimensional array" % other.ndim) @@ -135,7 +135,7 @@ def squared_distance(self, other): return result else: raise Exception("Cannot call squared_distance with %d-dimensional array" % - other.ndim) + other.ndim) else: result = 0.0 i, j = 0, 0 @@ -184,15 +184,14 @@ def __eq__(self, other): """ return (isinstance(other, self.__class__) - and other.size == self.size - and array_equal(other.indices, self.indices) - and array_equal(other.values, self.values)) + and other.size == self.size + and array_equal(other.indices, self.indices) + and array_equal(other.values, self.values)) def __ne__(self, other): return not self.__eq__(other) - class Vectors(object): """ Factory methods for working with vectors. Note that dense vectors diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index f4a83f0209e27..6c385042ffa5f 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -24,6 +24,7 @@ _serialize_tuple, RatingDeserializer from pyspark.rdd import RDD + class MatrixFactorizationModel(object): """A matrix factorisation model trained by regularized alternating least-squares. @@ -55,32 +56,34 @@ def predictAll(self, usersProducts): return RDD(self._java_model.predict(usersProductsJRDD._jrdd), self._context, RatingDeserializer()) + class ALS(object): @classmethod def train(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1): sc = ratings.context ratingBytes = _get_unmangled_rdd(ratings, _serialize_rating) - mod = sc._jvm.PythonMLLibAPI().trainALSModel(ratingBytes._jrdd, - rank, iterations, lambda_, blocks) + mod = sc._jvm.PythonMLLibAPI().trainALSModel( + ratingBytes._jrdd, rank, iterations, lambda_, blocks) return MatrixFactorizationModel(sc, mod) @classmethod def trainImplicit(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1, alpha=0.01): sc = ratings.context ratingBytes = _get_unmangled_rdd(ratings, _serialize_rating) - mod = sc._jvm.PythonMLLibAPI().trainImplicitALSModel(ratingBytes._jrdd, - rank, iterations, lambda_, blocks, alpha) + mod = sc._jvm.PythonMLLibAPI().trainImplicitALSModel( + ratingBytes._jrdd, rank, iterations, lambda_, blocks, alpha) return MatrixFactorizationModel(sc, mod) + def _test(): import doctest globs = globals().copy() globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) - (failure_count, test_count) = doctest.testmod(globs=globs, - optionflags=doctest.ELLIPSIS) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: exit(-1) + if __name__ == "__main__": _test() diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 266b31d3fab0e..bc7de6d2e8958 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -113,10 +113,9 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, initialWeights=None): """Train a linear regression model on the given data.""" sc = data.context - return _regression_train_wrapper(sc, lambda d, i: - sc._jvm.PythonMLLibAPI().trainLinearRegressionModelWithSGD( - d._jrdd, iterations, step, miniBatchFraction, i), - LinearRegressionModel, data, initialWeights) + train_f = lambda d, i: sc._jvm.PythonMLLibAPI().trainLinearRegressionModelWithSGD( + d._jrdd, iterations, step, miniBatchFraction, i) + return _regression_train_wrapper(sc, train_f, LinearRegressionModel, data, initialWeights) class LassoModel(LinearRegressionModelBase): @@ -157,10 +156,9 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): """Train a Lasso regression model on the given data.""" sc = data.context - return _regression_train_wrapper(sc, lambda d, i: - sc._jvm.PythonMLLibAPI().trainLassoModelWithSGD(d._jrdd, - iterations, step, regParam, miniBatchFraction, i), - LassoModel, data, initialWeights) + train_f = lambda d, i: sc._jvm.PythonMLLibAPI().trainLassoModelWithSGD( + d._jrdd, iterations, step, regParam, miniBatchFraction, i) + return _regression_train_wrapper(sc, train_f, LassoModel, data, initialWeights) class RidgeRegressionModel(LinearRegressionModelBase): @@ -201,18 +199,16 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): """Train a ridge regression model on the given data.""" sc = data.context - return _regression_train_wrapper(sc, lambda d, i: - sc._jvm.PythonMLLibAPI().trainRidgeModelWithSGD(d._jrdd, - iterations, step, regParam, miniBatchFraction, i), - RidgeRegressionModel, data, initialWeights) + train_func = lambda d, i: sc._jvm.PythonMLLibAPI().trainRidgeModelWithSGD( + d._jrdd, iterations, step, regParam, miniBatchFraction, i) + return _regression_train_wrapper(sc, train_func, RidgeRegressionModel, data, initialWeights) def _test(): import doctest globs = globals().copy() globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) - (failure_count, test_count) = doctest.testmod(globs=globs, - optionflags=doctest.ELLIPSIS) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 1ee96bb4af37b..37ccf1d590743 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -23,7 +23,7 @@ import unittest from pyspark.mllib._common import _convert_vector, _serialize_double_vector, \ - _deserialize_double_vector, _dot, _squared_distance + _deserialize_double_vector, _dot, _squared_distance from pyspark.mllib.linalg import SparseVector from pyspark.mllib.regression import LabeledPoint from pyspark.tests import PySparkTestCase @@ -46,12 +46,9 @@ def test_serialize(self): self.assertTrue(sv is _convert_vector(sv)) self.assertTrue(dv is _convert_vector(dv)) self.assertTrue(array_equal(dv, _convert_vector(lst))) - self.assertEquals(sv, - _deserialize_double_vector(_serialize_double_vector(sv))) - self.assertTrue(array_equal(dv, - _deserialize_double_vector(_serialize_double_vector(dv)))) - self.assertTrue(array_equal(dv, - _deserialize_double_vector(_serialize_double_vector(lst)))) + self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(sv))) + self.assertTrue(array_equal(dv, _deserialize_double_vector(_serialize_double_vector(dv)))) + self.assertTrue(array_equal(dv, _deserialize_double_vector(_serialize_double_vector(lst)))) def test_dot(self): sv = SparseVector(4, {1: 1, 3: 2}) @@ -132,7 +129,7 @@ def test_classification(self): def test_regression(self): from pyspark.mllib.regression import LinearRegressionWithSGD, LassoWithSGD, \ - RidgeRegressionWithSGD + RidgeRegressionWithSGD data = [ LabeledPoint(-1.0, [0, -1]), LabeledPoint(1.0, [0, 1]), @@ -179,14 +176,10 @@ def test_serialize(self): self.assertEquals(sv, _convert_vector(lil.tocoo())) self.assertEquals(sv, _convert_vector(lil.tocsr())) self.assertEquals(sv, _convert_vector(lil.todok())) - self.assertEquals(sv, - _deserialize_double_vector(_serialize_double_vector(lil))) - self.assertEquals(sv, - _deserialize_double_vector(_serialize_double_vector(lil.tocsc()))) - self.assertEquals(sv, - _deserialize_double_vector(_serialize_double_vector(lil.tocsr()))) - self.assertEquals(sv, - _deserialize_double_vector(_serialize_double_vector(lil.todok()))) + self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(lil))) + self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(lil.tocsc()))) + self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(lil.tocsr()))) + self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(lil.todok()))) def test_dot(self): from scipy.sparse import lil_matrix @@ -265,7 +258,7 @@ def test_classification(self): def test_regression(self): from pyspark.mllib.regression import LinearRegressionWithSGD, LassoWithSGD, \ - RidgeRegressionWithSGD + RidgeRegressionWithSGD data = [ LabeledPoint(-1.0, self.scipy_matrix(2, {1: -1.0})), LabeledPoint(1.0, self.scipy_matrix(2, {1: 1.0})), diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 50d0cdd087625..0e5f4520b9402 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -21,6 +21,7 @@ from pyspark.mllib.regression import LabeledPoint from pyspark.mllib._common import _convert_vector + class MLUtils: """ Helper methods to load, save and pre-process data used in MLlib. @@ -44,7 +45,6 @@ def _parse_libsvm_line(line, multiclass): values[i] = float(value) return label, indices, values - @staticmethod def _convert_labeled_point_to_libsvm(p): """Converts a LabeledPoint to a string in LIBSVM format.""" @@ -62,7 +62,6 @@ def _convert_labeled_point_to_libsvm(p): " but got " % type(v)) return " ".join(items) - @staticmethod def loadLibSVMFile(sc, path, multiclass=False, numFeatures=-1, minPartitions=None): """ @@ -135,7 +134,6 @@ def loadLibSVMFile(sc, path, multiclass=False, numFeatures=-1, minPartitions=Non numFeatures = parsed.map(lambda x: 0 if x[1].size == 0 else x[1][-1]).reduce(max) + 1 return parsed.map(lambda x: LabeledPoint(x[0], Vectors.sparse(numFeatures, x[1], x[2]))) - @staticmethod def saveAsLibSVMFile(data, dir): """ From 0659529614c804e0c04efc59cb67dab3a6cdc9d9 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Sun, 25 May 2014 17:15:47 -0700 Subject: [PATCH 425/641] SPARK-1903 Document Spark's network connections https://issues.apache.org/jira/browse/SPARK-1903 Author: Andrew Ash Closes #856 from ash211/SPARK-1903 and squashes the following commits: 6e7782a [Andrew Ash] Add the technology used on each port 1d9b5d3 [Andrew Ash] Document port for history server 56193ee [Andrew Ash] spark.ui.port becomes worker.ui.port and master.ui.port a774c07 [Andrew Ash] Wording in network section 90e8237 [Andrew Ash] Use real :toc instead of the hand-written one edaa337 [Andrew Ash] Master -> Standalone Cluster Master 57e8869 [Andrew Ash] Port -> Default Port 3d4d289 [Andrew Ash] Title to title case c7d42d9 [Andrew Ash] [WIP] SPARK-1903 Add initial port listing for documentation a416ae9 [Andrew Ash] Word wrap to 100 lines --- docs/README.md | 43 +++++-- docs/configuration.md | 268 ++++++++++++++++++++++++++++++------------ 2 files changed, 222 insertions(+), 89 deletions(-) diff --git a/docs/README.md b/docs/README.md index f1eb644f93406..fd7ba4e0d72ea 100644 --- a/docs/README.md +++ b/docs/README.md @@ -1,23 +1,31 @@ Welcome to the Spark documentation! -This readme will walk you through navigating and building the Spark documentation, which is included here with the Spark source code. You can also find documentation specific to release versions of Spark at http://spark.apache.org/documentation.html. +This readme will walk you through navigating and building the Spark documentation, which is included +here with the Spark source code. You can also find documentation specific to release versions of +Spark at http://spark.apache.org/documentation.html. -Read on to learn more about viewing documentation in plain text (i.e., markdown) or building the documentation yourself. Why build it yourself? So that you have the docs that corresponds to whichever version of Spark you currently have checked out of revision control. +Read on to learn more about viewing documentation in plain text (i.e., markdown) or building the +documentation yourself. Why build it yourself? So that you have the docs that corresponds to +whichever version of Spark you currently have checked out of revision control. ## Generating the Documentation HTML -We include the Spark documentation as part of the source (as opposed to using a hosted wiki, such as the github wiki, as the definitive documentation) to enable the documentation to evolve along with the source code and be captured by revision control (currently git). This way the code automatically includes the version of the documentation that is relevant regardless of which version or release you have checked out or downloaded. +We include the Spark documentation as part of the source (as opposed to using a hosted wiki, such as +the github wiki, as the definitive documentation) to enable the documentation to evolve along with +the source code and be captured by revision control (currently git). This way the code automatically +includes the version of the documentation that is relevant regardless of which version or release +you have checked out or downloaded. -In this directory you will find textfiles formatted using Markdown, with an ".md" suffix. You can read those text files directly if you want. Start with index.md. +In this directory you will find textfiles formatted using Markdown, with an ".md" suffix. You can +read those text files directly if you want. Start with index.md. -The markdown code can be compiled to HTML using the -[Jekyll tool](http://jekyllrb.com). +The markdown code can be compiled to HTML using the [Jekyll tool](http://jekyllrb.com). To use the `jekyll` command, you will need to have Jekyll installed. The easiest way to do this is via a Ruby Gem, see the [jekyll installation instructions](http://jekyllrb.com/docs/installation). If not already installed, you need to install `kramdown` with `sudo gem install kramdown`. -Execute `jekyll` from the `docs/` directory. Compiling the site with Jekyll will create a directory called -`_site` containing index.html as well as the rest of the compiled files. +Execute `jekyll` from the `docs/` directory. Compiling the site with Jekyll will create a directory +called `_site` containing index.html as well as the rest of the compiled files. You can modify the default Jekyll build as follows: @@ -30,9 +38,11 @@ You can modify the default Jekyll build as follows: ## Pygments -We also use pygments (http://pygments.org) for syntax highlighting in documentation markdown pages, so you will also need to install that (it requires Python) by running `sudo easy_install Pygments`. +We also use pygments (http://pygments.org) for syntax highlighting in documentation markdown pages, +so you will also need to install that (it requires Python) by running `sudo easy_install Pygments`. -To mark a block of code in your markdown to be syntax highlighted by jekyll during the compile phase, use the following sytax: +To mark a block of code in your markdown to be syntax highlighted by jekyll during the compile +phase, use the following sytax: {% highlight scala %} // Your scala code goes here, you can replace scala with many other @@ -43,8 +53,15 @@ To mark a block of code in your markdown to be syntax highlighted by jekyll duri You can build just the Spark scaladoc by running `sbt/sbt doc` from the SPARK_PROJECT_ROOT directory. -Similarly, you can build just the PySpark epydoc by running `epydoc --config epydoc.conf` from the SPARK_PROJECT_ROOT/pyspark directory. Documentation is only generated for classes that are listed as public in `__init__.py`. +Similarly, you can build just the PySpark epydoc by running `epydoc --config epydoc.conf` from the +SPARK_PROJECT_ROOT/pyspark directory. Documentation is only generated for classes that are listed as +public in `__init__.py`. -When you run `jekyll` in the `docs` directory, it will also copy over the scaladoc for the various Spark subprojects into the `docs` directory (and then also into the `_site` directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/). +When you run `jekyll` in the `docs` directory, it will also copy over the scaladoc for the various +Spark subprojects into the `docs` directory (and then also into the `_site` directory). We use a +jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it +may take some time as it generates all of the scaladoc. The jekyll plugin also generates the +PySpark docs using [epydoc](http://epydoc.sourceforge.net/). -NOTE: To skip the step of building and copying over the Scala and Python API docs, run `SKIP_API=1 jekyll`. +NOTE: To skip the step of building and copying over the Scala and Python API docs, run `SKIP_API=1 +jekyll`. diff --git a/docs/configuration.md b/docs/configuration.md index 4d41c36e38e26..e5d955f23fe32 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -3,15 +3,10 @@ layout: global title: Spark Configuration --- -Spark provides three locations to configure the system: - -* [Spark properties](#spark-properties) control most application parameters and can be set by - passing a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object to SparkContext, - or through the `conf/spark-defaults.conf` properties file. -* [Environment variables](#environment-variables) can be used to set per-machine settings, such as - the IP address, through the `conf/spark-env.sh` script on each node. -* [Logging](#configuring-logging) can be configured through `log4j.properties`. +* This will become a table of contents (this text will be scraped). +{:toc} +Spark provides several locations to configure the system: # Spark Properties @@ -65,7 +60,8 @@ there are at least five properties that you will commonly want to control:
      spark.executor.memory 512m - Amount of memory to use per executor process, in the same format as JVM memory strings (e.g. 512m, 2g). + Amount of memory to use per executor process, in the same format as JVM memory strings (e.g. + 512m, 2g).
      spark.local.dir /tmp - Directory to use for "scratch" space in Spark, including map output files and RDDs that get stored - on disk. This should be on a fast, local disk in your system. It can also be a comma-separated - list of multiple directories on different disks. + Directory to use for "scratch" space in Spark, including map output files and RDDs that get + stored on disk. This should be on a fast, local disk in your system. It can also be a + comma-separated list of multiple directories on different disks. NOTE: In Spark 1.0 and later this will be overriden by SPARK_LOCAL_DIRS (Standalone, Mesos) or LOCAL_DIRS (YARN) envrionment variables set by the cluster manager. @@ -130,8 +126,8 @@ Apart from these, the following properties are also available, and may be useful - Default number of tasks to use across the cluster for distributed shuffle operations (groupByKey, - reduceByKey, etc) when not set by user. + Default number of tasks to use across the cluster for distributed shuffle operations + (groupByKey, reduceByKey, etc) when not set by user.
      spark.tachyonStore.baseDir System.getProperty("java.io.tmpdir") - Directories of the Tachyon File System that store RDDs. The Tachyon file system's URL is set by spark.tachyonStore.url. - It can also be a comma-separated list of multiple directories on Tachyon file system. + Directories of the Tachyon File System that store RDDs. The Tachyon file system's URL is set by + spark.tachyonStore.url. It can also be a comma-separated list of multiple + directories on Tachyon file system.
      spark.mesos.coarse false - If set to "true", runs over Mesos clusters in - "coarse-grained" sharing mode, - where Spark acquires one long-lived Mesos task on each machine instead of one Mesos task per Spark task. + If set to "true", runs over Mesos clusters in "coarse-grained" sharing mode, where Spark + acquires one long-lived Mesos task on each machine instead of one Mesos task per Spark task. This gives lower-latency scheduling for short queries, but leaves resources in use for the whole duration of the Spark job. spark.io.compression.codec org.apache.spark.io.
      LZFCompressionCodec
      - The codec used to compress internal data such as RDD partitions and shuffle outputs. By default, Spark provides two - codecs: org.apache.spark.io.LZFCompressionCodec and org.apache.spark.io.SnappyCompressionCodec. + The codec used to compress internal data such as RDD partitions and shuffle outputs. By default, + Spark provides two codecs: org.apache.spark.io.LZFCompressionCodec and + org.apache.spark.io.SnappyCompressionCodec.
      spark.io.compression.snappy.block.size 32768 - Block size (in bytes) used in Snappy compression, in the case when Snappy compression codec is used. + Block size (in bytes) used in Snappy compression, in the case when Snappy compression codec is + used.
      spark.scheduler.revive.interval 1000 - The interval length for the scheduler to revive the worker resource offers to run tasks. (in milliseconds) + The interval length for the scheduler to revive the worker resource offers to run tasks. (in + milliseconds)
      48 Maximum size (in megabytes) of map outputs to fetch simultaneously from each reduce task. Since - each output requires us to create a buffer to receive it, this represents a fixed memory overhead - per reduce task, so keep it small unless you have a large amount of memory. + each output requires us to create a buffer to receive it, this represents a fixed memory + overhead per reduce task, so keep it small unless you have a large amount of memory.
      spark.kryoserializer.buffer.mb 2 - Maximum object size to allow within Kryo (the library needs to create a buffer at least as - large as the largest single object you'll serialize). Increase this if you get a "buffer limit - exceeded" exception inside Kryo. Note that there will be one buffer per core on each worker. + Maximum object size to allow within Kryo (the library needs to create a buffer at least as large + as the largest single object you'll serialize). Increase this if you get a "buffer limit + exceeded" exception inside Kryo. Note that there will be one buffer per core on each + worker.
      spark.worker.cleanup.enabled false - Enable periodic cleanup of worker / application directories. Note that this only affects standalone - mode, as YARN works differently. Applications directories are cleaned up regardless of whether - the application is still running. + Enable periodic cleanup of worker / application directories. Note that this only affects + standalone mode, as YARN works differently. Applications directories are cleaned up regardless + of whether the application is still running.
      spark.worker.cleanup.appDataTtl 7 * 24 * 3600 (7 days) - The number of seconds to retain application work directories on each worker. This is a Time To Live - and should depend on the amount of available disk space you have. Application logs and jars are - downloaded to each application work dir. Over time, the work dirs can quickly fill up disk space, - especially if you run jobs very frequently. + The number of seconds to retain application work directories on each worker. This is a Time To + Live and should depend on the amount of available disk space you have. Application logs and + jars are downloaded to each application work dir. Over time, the work dirs can quickly fill up + disk space, especially if you run jobs very frequently.
      spark.akka.heartbeat.pauses 600 - This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart beat pause in seconds for akka. This can be used to control sensitivity to gc pauses. Tune this in combination of `spark.akka.heartbeat.interval` and `spark.akka.failure-detector.threshold` if you need to. + This is set to a larger value to disable failure detector that comes inbuilt akka. It can be + enabled again, if you plan to use this feature (Not recommended). Acceptable heart beat pause + in seconds for akka. This can be used to control sensitivity to gc pauses. Tune this in + combination of `spark.akka.heartbeat.interval` and `spark.akka.failure-detector.threshold` if + you need to.
      spark.akka.failure-detector.threshold 300.0 - This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). This maps to akka's `akka.remote.transport-failure-detector.threshold`. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.heartbeat.interval` if you need to. + This is set to a larger value to disable failure detector that comes inbuilt akka. It can be + enabled again, if you plan to use this feature (Not recommended). This maps to akka's + `akka.remote.transport-failure-detector.threshold`. Tune this in combination of + `spark.akka.heartbeat.pauses` and `spark.akka.heartbeat.interval` if you need to.
      spark.akka.heartbeat.interval 1000 - This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using failure detector can be, a sensistive failure detector can help evict rogue executors really quick. However this is usually not the case as gc pauses and network lags are expected in a real spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats between nodes leading to flooding the network with those. + This is set to a larger value to disable failure detector that comes inbuilt akka. It can be + enabled again, if you plan to use this feature (Not recommended). A larger interval value in + seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for + akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and + `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using failure + detector can be, a sensistive failure detector can help evict rogue executors really quick. + However this is usually not the case as gc pauses and network lags are expected in a real spark + cluster. Apart from that enabling this leads to a lot of exchanges of heart beats between nodes + leading to flooding the network with those.
      spark.cleaner.ttl (infinite) - Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks generated, etc.). - Periodic cleanups will ensure that metadata older than this duration will be forgotten. This is - useful for running Spark for many hours / days (for example, running 24/7 in case of Spark Streaming - applications). Note that any RDD that persists in memory for more than this duration will be cleared as well. + Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks + generated, etc.). Periodic cleanups will ensure that metadata older than this duration will be + forgotten. This is useful for running Spark for many hours / days (for example, running 24/7 in + case of Spark Streaming applications). Note that any RDD that persists in memory for more than + this duration will be cleared as well.
      4096 Size of each piece of a block in kilobytes for TorrentBroadcastFactory. - Too large a value decreases parallelism during broadcast (makes it slower); however, if it is too small, BlockManager might take a performance hit. + Too large a value decreases parallelism during broadcast (makes it slower); however, if it is + too small, BlockManager might take a performance hit.
      spark.shuffle.consolidateFiles false - If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files can improve filesystem performance for shuffles with large numbers of reduce tasks. It is recommended to set this to "true" when using ext4 or xfs filesystems. On ext3, this option might degrade performance on machines with many (>8) cores due to filesystem limitations. + If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files + can improve filesystem performance for shuffles with large numbers of reduce tasks. It is + recommended to set this to "true" when using ext4 or xfs filesystems. On ext3, this option might + degrade performance on machines with many (>8) cores due to filesystem limitations.
      spark.shuffle.spill true - If set to "true", limits the amount of memory used during reduces by spilling data out to disk. This spilling - threshold is specified by spark.shuffle.memoryFraction. + If set to "true", limits the amount of memory used during reduces by spilling data out to disk. + This spilling threshold is specified by spark.shuffle.memoryFraction.
      spark.speculation false - If set to "true", performs speculative execution of tasks. This means if one or more tasks are running slowly in a stage, they will be re-launched. + If set to "true", performs speculative execution of tasks. This means if one or more tasks are + running slowly in a stage, they will be re-launched.
      spark.eventLog.enabled false - Whether to log spark events, useful for reconstructing the Web UI after the application has finished. + Whether to log spark events, useful for reconstructing the Web UI after the application has + finished.
      file:///tmp/spark-events Base directory in which spark events are logged, if spark.eventLog.enabled is true. - Within this base directory, Spark creates a sub-directory for each application, and logs the events - specific to the application in this directory. + Within this base directory, Spark creates a sub-directory for each application, and logs the + events specific to the application in this directory.
      spark.deploy.spreadOut true - Whether the standalone cluster manager should spread applications out across nodes or try - to consolidate them onto as few nodes as possible. Spreading out is usually better for - data locality in HDFS, but consolidating is more efficient for compute-intensive workloads.
      - Note: this setting needs to be configured in the standalone cluster master, not in individual - applications; you can set it through SPARK_MASTER_OPTS in spark-env.sh. + Whether the standalone cluster manager should spread applications out across nodes or try to + consolidate them onto as few nodes as possible. Spreading out is usually better for data + locality in HDFS, but consolidating is more efficient for compute-intensive workloads.
      + Note: this setting needs to be configured in the standalone cluster master, not in + individual applications; you can set it through SPARK_MASTER_OPTS in + spark-env.sh.
      spark.deploy.defaultCores (infinite) - Default number of cores to give to applications in Spark's standalone mode if they don't - set spark.cores.max. If not set, applications always get all available - cores unless they configure spark.cores.max themselves. - Set this lower on a shared cluster to prevent users from grabbing - the whole cluster by default.
      - Note: this setting needs to be configured in the standalone cluster master, not in individual - applications; you can set it through SPARK_MASTER_OPTS in spark-env.sh. + Default number of cores to give to applications in Spark's standalone mode if they don't set + spark.cores.max. If not set, applications always get all available cores unless + they configure spark.cores.max themselves. Set this lower on a shared cluster to + prevent users from grabbing the whole cluster by default.
      Note: this setting needs + to be configured in the standalone cluster master, not in individual applications; you can set + it through SPARK_MASTER_OPTS in spark-env.sh.
      spark.files.overwrite false - Whether to overwrite files added through SparkContext.addFile() when the target file exists and its contents do not match those of the source. + Whether to overwrite files added through SparkContext.addFile() when the target file exists and + its contents do not match those of the source.
      spark.authenticate false - Whether spark authenticates its internal connections. See spark.authenticate.secret if not - running on Yarn. + Whether spark authenticates its internal connections. See spark.authenticate.secret + if not running on Yarn.
      -In addition to the above, there are also options for setting up the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores to use on each machine and maximum memory. +In addition to the above, there are also options for setting up the Spark [standalone cluster +scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores to use on each +machine and maximum memory. -Since `spark-env.sh` is a shell script, some of these can be set programmatically -- for example, you might -compute `SPARK_LOCAL_IP` by looking up the IP of a specific network interface. +Since `spark-env.sh` is a shell script, some of these can be set programmatically -- for example, +you might compute `SPARK_LOCAL_IP` by looking up the IP of a specific network interface. # Configuring Logging -Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can configure it by adding a `log4j.properties` -file in the `conf` directory. One way to start is to copy the existing `log4j.properties.template` located there. +Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can configure it by adding a +`log4j.properties` file in the `conf` directory. One way to start is to copy the existing +`log4j.properties.template` located there. + +# Configuring Ports for Network Security + +Spark makes heavy use of the network, and some environments have strict requirements for using tight +firewall settings. Below are the primary ports that Spark uses for its communication and how to +configure those ports. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      FromToDefault PortPurposeConfiguration + SettingNotes
      BrowserStandalone Cluster Master8080Web UImaster.ui.portJetty-based
      BrowserWorker8081Web UIworker.ui.portJetty-based
      BrowserDriver4040Web UIspark.ui.portJetty-based
      BrowserHistory Server18080Web UIspark.history.ui.portJetty-based
      ApplicationStandalone Cluster Master7077Submit job to clusterspark.driver.portAkka-based. Set to "0" to choose a port randomly
      WorkerStandalone Cluster Master7077Join clusterspark.driver.portAkka-based. Set to "0" to choose a port randomly
      ApplicationWorker(random)Join clusterSPARK_WORKER_PORT (standalone cluster)Akka-based
      Driver and other WorkersWorker(random) +
        +
      • File server for file and jars
      • +
      • Http Broadcast
      • +
      • Class file server (Spark Shell only)
      • +
      +
      NoneJetty-based. Each of these services starts on a random port that cannot be configured
      From c3576ffcd7910e38928f233a824dd9e037cde05f Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sun, 25 May 2014 18:37:44 -0700 Subject: [PATCH 426/641] [SQL] Minor: Introduce SchemaRDD#aggregate() for simple aggregations ```scala rdd.aggregate(Sum('val)) ``` is just shorthand for ```scala rdd.groupBy()(Sum('val)) ``` but seems be more natural than doing a groupBy with no grouping expressions when you really just want an aggregation over all rows. Did not add a JavaSchemaRDD or Python API, as these seem to be lacking several other methods like groupBy() already -- leaving that cleanup for future patches. Author: Aaron Davidson Closes #874 from aarondav/schemardd and squashes the following commits: e9e68ee [Aaron Davidson] Add comment db6afe2 [Aaron Davidson] Introduce SchemaRDD#aggregate() for simple aggregations --- .../scala/org/apache/spark/sql/SchemaRDD.scala | 18 ++++++++++++++++-- .../org/apache/spark/sql/DslQuerySuite.scala | 8 ++++++++ 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 9883ebc0b3c62..e855f36256bc5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -59,7 +59,7 @@ import java.util.{Map => JMap} * // Importing the SQL context gives access to all the SQL functions and implicit conversions. * import sqlContext._ * - * val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_\$i"))) + * val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) * // Any RDD containing case classes can be registered as a table. The schema of the table is * // automatically inferred using scala reflection. * rdd.registerAsTable("records") @@ -204,6 +204,20 @@ class SchemaRDD( new SchemaRDD(sqlContext, Aggregate(groupingExprs, aliasedExprs, logicalPlan)) } + /** + * Performs an aggregation over all Rows in this RDD. + * This is equivalent to a groupBy with no grouping expressions. + * + * {{{ + * schemaRDD.aggregate(Sum('sales) as 'totalSales) + * }}} + * + * @group Query + */ + def aggregate(aggregateExprs: Expression*): SchemaRDD = { + groupBy()(aggregateExprs: _*) + } + /** * Applies a qualifier to the attributes of this relation. Can be used to disambiguate attributes * with the same name, for example, when performing self-joins. @@ -281,7 +295,7 @@ class SchemaRDD( * supports features such as filter pushdown. */ @Experimental - override def count(): Long = groupBy()(Count(Literal(1))).collect().head.getLong(0) + override def count(): Long = aggregate(Count(Literal(1))).collect().head.getLong(0) /** * :: Experimental :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 94ba13b14b33d..692569a73ffcf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -39,6 +39,14 @@ class DslQuerySuite extends QueryTest { testData2.groupBy('a)('a, Sum('b)), Seq((1,3),(2,3),(3,3)) ) + checkAnswer( + testData2.groupBy('a)('a, Sum('b) as 'totB).aggregate(Sum('totB)), + 9 + ) + checkAnswer( + testData2.aggregate(Sum('b)), + 9 + ) } test("select *") { From b6d22af040073cd611b0fcfdf8a5259c0dfd854c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 25 May 2014 20:13:32 -0700 Subject: [PATCH 427/641] HOTFIX: Add no-arg SparkContext constructor in Java Self explanatory. Author: Patrick Wendell Closes #878 from pwendell/java-constructor and squashes the following commits: 2cc1605 [Patrick Wendell] HOTFIX: Add no-arg SparkContext constructor in Java --- .../scala/org/apache/spark/api/java/JavaSparkContext.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index a7cfee6d01711..1e0493c4855e0 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -41,6 +41,12 @@ import org.apache.spark.rdd.RDD * [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones. */ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWorkaround { + /** + * Create a JavaSparkContext that loads settings from system properties (for instance, when + * launching with ./bin/spark-submit). + */ + def this() = this(new SparkContext()) + /** * @param conf a [[org.apache.spark.SparkConf]] object specifying Spark parameters */ From d6395d86f90d1c47c5b6ad17c618b56e00b7fc85 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 26 May 2014 00:17:20 -0700 Subject: [PATCH 428/641] [SPARK-1914] [SQL] Simplify CountFunction not to traverse to evaluate all child expressions. `CountFunction` should count up only if the child's evaluated value is not null. Because it traverses to evaluate all child expressions, even if the child is null, it counts up if one of the all children is not null. Author: Takuya UESHIN Closes #861 from ueshin/issues/SPARK-1914 and squashes the following commits: 3b37315 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-1914 2afa238 [Takuya UESHIN] Simplify CountFunction not to traverse to evaluate all child expressions. --- .../apache/spark/sql/catalyst/expressions/aggregates.scala | 4 ++-- .../src/test/scala/org/apache/spark/sql/DslQuerySuite.scala | 5 +++++ 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 1bcd4e22766a9..79937b129aeae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -298,8 +298,8 @@ case class CountFunction(expr: Expression, base: AggregateExpression) extends Ag var count: Long = _ override def update(input: Row): Unit = { - val evaluatedExpr = expr.map(_.eval(input)) - if (evaluatedExpr.map(_ != null).reduceLeft(_ || _)) { + val evaluatedExpr = expr.eval(input) + if (evaluatedExpr != null) { count += 1L } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 692569a73ffcf..8197e8a18d447 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -125,6 +125,11 @@ class DslQuerySuite extends QueryTest { Seq((1,0), (2, 1)) ) + checkAnswer( + testData3.groupBy('a)('a, Count('a + 'b)), + Seq((1,0), (2, 1)) + ) + checkAnswer( testData3.groupBy()(Count('a), Count('b), Count(1), CountDistinct('a :: Nil), CountDistinct('b :: Nil)), (2, 1, 2, 2, 1) :: Nil From bee6c4f4a155f625495212c17b58dc76f525f312 Mon Sep 17 00:00:00 2001 From: witgo Date: Mon, 26 May 2014 13:16:35 -0700 Subject: [PATCH 429/641] Fix scalastyle warnings in yarn alpha Author: witgo Closes #884 from witgo/scalastyle and squashes the following commits: 4b08ae4 [witgo] Fix scalastyle warnings in yarn alpha --- dev/scalastyle | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dev/scalastyle b/dev/scalastyle index a972811ba8ed6..0e8fd5cc8d64c 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -19,7 +19,8 @@ echo -e "q\n" | SPARK_HIVE=true sbt/sbt scalastyle > scalastyle.txt # Check style with YARN alpha built too -echo -e "q\n" | SPARK_YARN=true sbt/sbt yarn/scalastyle >> scalastyle.txt +echo -e "q\n" | SPARK_HADOOP_VERSION=0.23.9 SPARK_YARN=true sbt/sbt yarn-alpha/scalastyle \ + >> scalastyle.txt # Check style with YARN built too echo -e "q\n" | SPARK_HADOOP_VERSION=2.2.0 SPARK_YARN=true sbt/sbt yarn/scalastyle \ >> scalastyle.txt From cb7fe5034826844f1b50fbe8b92646317b66f21c Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 26 May 2014 14:34:58 -0700 Subject: [PATCH 430/641] SPARK-1925: Replace '&' with '&&' JIRA: https://issues.apache.org/jira/browse/SPARK-1925 Author: zsxwing Closes #879 from zsxwing/SPARK-1925 and squashes the following commits: 5cf5a6d [zsxwing] SPARK-1925: Replace '&' with '&&' --- .../main/scala/org/apache/spark/mllib/tree/DecisionTree.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 0fe30a3e7040b..3b13e52a7b445 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -401,7 +401,7 @@ object DecisionTree extends Serializable with Logging { */ def isSampleValid(parentFilters: List[Filter], labeledPoint: LabeledPoint): Boolean = { // leaf - if ((level > 0) & (parentFilters.length == 0)) { + if ((level > 0) && (parentFilters.length == 0)) { return false } @@ -454,7 +454,7 @@ object DecisionTree extends Serializable with Logging { val bin = binForFeatures(mid) val lowThreshold = bin.lowSplit.threshold val highThreshold = bin.highSplit.threshold - if ((lowThreshold < feature) & (highThreshold >= feature)){ + if ((lowThreshold < feature) && (highThreshold >= feature)){ return mid } else if (lowThreshold >= feature) { From 56c771cb2d00a5843c391ae6561536ee46e535d4 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 26 May 2014 16:10:22 -0700 Subject: [PATCH 431/641] [SPARK-1931] Reconstruct routing tables in Graph.partitionBy 905173df57b90f90ebafb22e43f55164445330e6 introduced a bug in partitionBy where, after repartitioning the edges, it reuses the VertexRDD without updating the routing tables to reflect the new edge layout. Subsequent accesses of the triplets contain nulls for many vertex properties. This commit adds a test for this bug and fixes it by introducing `VertexRDD#withEdges` and calling it in `partitionBy`. Author: Ankur Dave Closes #885 from ankurdave/SPARK-1931 and squashes the following commits: 3930cdd [Ankur Dave] Note how to set up VertexRDD for efficient joins 9bdbaa4 [Ankur Dave] [SPARK-1931] Reconstruct routing tables in Graph.partitionBy --- .../scala/org/apache/spark/graphx/VertexRDD.scala | 12 ++++++++++++ .../org/apache/spark/graphx/impl/GraphImpl.scala | 13 +++++++++---- .../scala/org/apache/spark/graphx/GraphSuite.scala | 10 ++++++++++ 3 files changed, 31 insertions(+), 4 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index 8c62897037b6d..8b910fbc5a423 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -300,6 +300,18 @@ class VertexRDD[@specialized VD: ClassTag]( def reverseRoutingTables(): VertexRDD[VD] = this.mapVertexPartitions(vPart => vPart.withRoutingTable(vPart.routingTable.reverse)) + /** Prepares this VertexRDD for efficient joins with the given EdgeRDD. */ + def withEdges(edges: EdgeRDD[_, _]): VertexRDD[VD] = { + val routingTables = VertexRDD.createRoutingTables(edges, this.partitioner.get) + val vertexPartitions = partitionsRDD.zipPartitions(routingTables, true) { + (partIter, routingTableIter) => + val routingTable = + if (routingTableIter.hasNext) routingTableIter.next() else RoutingTablePartition.empty + partIter.map(_.withRoutingTable(routingTable)) + } + new VertexRDD(vertexPartitions) + } + /** Generates an RDD of vertex attributes suitable for shipping to the edge partitions. */ private[graphx] def shipVertexAttributes( shipSrc: Boolean, shipDst: Boolean): RDD[(PartitionID, VertexAttributeBlock[VD])] = { 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 2f2d0e03fd7b5..1649b244d2881 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 @@ -88,8 +88,8 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } val edgePartition = builder.toEdgePartition Iterator((pid, edgePartition)) - }, preservesPartitioning = true)) - GraphImpl.fromExistingRDDs(vertices, newEdges) + }, preservesPartitioning = true)).cache() + GraphImpl.fromExistingRDDs(vertices.withEdges(newEdges), newEdges) } override def reverse: Graph[VD, ED] = { @@ -277,7 +277,11 @@ object GraphImpl { GraphImpl(vertexRDD, edgeRDD) } - /** Create a graph from a VertexRDD and an EdgeRDD with arbitrary replicated vertices. */ + /** + * Create a graph from a VertexRDD and an EdgeRDD with arbitrary replicated vertices. The + * VertexRDD must already be set up for efficient joins with the EdgeRDD by calling + * `VertexRDD.withEdges` or an appropriate VertexRDD constructor. + */ def apply[VD: ClassTag, ED: ClassTag]( vertices: VertexRDD[VD], edges: EdgeRDD[ED, _]): GraphImpl[VD, ED] = { @@ -290,7 +294,8 @@ object GraphImpl { /** * Create a graph from a VertexRDD and an EdgeRDD with the same replicated vertex type as the - * vertices. + * vertices. The VertexRDD must already be set up for efficient joins with the EdgeRDD by calling + * `VertexRDD.withEdges` or an appropriate VertexRDD constructor. */ def fromExistingRDDs[VD: ClassTag, ED: ClassTag]( vertices: VertexRDD[VD], diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index 7b9bac5d9c8ea..abc25d0671133 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -133,6 +133,16 @@ class GraphSuite extends FunSuite with LocalSparkContext { Iterator((part.srcIds ++ part.dstIds).toSet) }.collect assert(verts.exists(id => partitionSetsUnpartitioned.count(_.contains(id)) > bound)) + + // Forming triplets view + val g = Graph( + sc.parallelize(List((0L, "a"), (1L, "b"), (2L, "c"))), + sc.parallelize(List(Edge(0L, 1L, 1), Edge(0L, 2L, 1)), 2)) + assert(g.triplets.collect.map(_.toTuple).toSet === + Set(((0L, "a"), (1L, "b"), 1), ((0L, "a"), (2L, "c"), 1))) + val gPart = g.partitionBy(EdgePartition2D) + assert(gPart.triplets.collect.map(_.toTuple).toSet === + Set(((0L, "a"), (1L, "b"), 1), ((0L, "a"), (2L, "c"), 1))) } } From 8d271c90fa496cb24e2b7362ef0497563591b97d Mon Sep 17 00:00:00 2001 From: Zhen Peng Date: Mon, 26 May 2014 21:30:25 -0700 Subject: [PATCH 432/641] SPARK-1929 DAGScheduler suspended by local task OOM DAGScheduler does not handle local task OOM properly, and will wait for the job result forever. Author: Zhen Peng Closes #883 from zhpengg/bugfix-dag-scheduler-oom and squashes the following commits: 76f7eda [Zhen Peng] remove redundant memory allocations aa63161 [Zhen Peng] SPARK-1929 DAGScheduler suspended by local task OOM --- .../org/apache/spark/scheduler/DAGScheduler.scala | 6 +++++- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 14 ++++++++++++++ 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ff411e24a3d85..c70aa0e6e4523 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import java.io.NotSerializableException +import java.io.{NotSerializableException, PrintWriter, StringWriter} import java.util.Properties import java.util.concurrent.atomic.AtomicInteger @@ -580,6 +580,10 @@ class DAGScheduler( case e: Exception => jobResult = JobFailed(e) job.listener.jobFailed(e) + case oom: OutOfMemoryError => + val exception = new SparkException("job failed for Out of memory exception", oom) + jobResult = JobFailed(exception) + job.listener.jobFailed(exception) } finally { val s = job.finalStage stageIdToJobIds -= s.id // clean up data structures that were populated for a local job, diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index d172dd1ac8e1b..81e64c1846ed5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -256,6 +256,20 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assertDataStructuresEmpty } + test("local job oom") { + val rdd = new MyRDD(sc, Nil) { + override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = + throw new java.lang.OutOfMemoryError("test local job oom") + override def getPartitions = Array( new Partition { override def index = 0 } ) + override def getPreferredLocations(split: Partition) = Nil + override def toString = "DAGSchedulerSuite Local RDD" + } + val jobId = scheduler.nextJobId.getAndIncrement() + runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, null, jobListener)) + assert(results.size == 0) + assertDataStructuresEmpty + } + test("run trivial job w/ dependency") { val baseRdd = makeRdd(1, Nil) val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) From ef690e1f69cb8e2e03bb0c43e3ccb2c54c995df7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 26 May 2014 21:31:27 -0700 Subject: [PATCH 433/641] Fixed the error message for OutOfMemoryError in DAGScheduler. --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c70aa0e6e4523..ccff6a3d1aebc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -581,7 +581,7 @@ class DAGScheduler( jobResult = JobFailed(e) job.listener.jobFailed(e) case oom: OutOfMemoryError => - val exception = new SparkException("job failed for Out of memory exception", oom) + val exception = new SparkException("Local job aborted due to out of memory error", oom) jobResult = JobFailed(exception) job.listener.jobFailed(exception) } finally { From 9ed37190f45fd9e6aa0f2c73b66d317732a53eb8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 26 May 2014 21:40:52 -0700 Subject: [PATCH 434/641] Updated dev Python scripts to make them PEP8 compliant. Author: Reynold Xin Closes #875 from rxin/pep8-dev-scripts and squashes the following commits: 04b084f [Reynold Xin] Made dev Python scripts PEP8 compliant. --- dev/audit-release/audit_release.py | 225 ++++++------ dev/create-release/generate-changelist.py | 160 ++++----- dev/merge_spark_pr.py | 402 +++++++++++----------- 3 files changed, 408 insertions(+), 379 deletions(-) diff --git a/dev/audit-release/audit_release.py b/dev/audit-release/audit_release.py index 8c7573b91f688..230e900ecd4de 100755 --- a/dev/audit-release/audit_release.py +++ b/dev/audit-release/audit_release.py @@ -30,18 +30,18 @@ import time import urllib2 -## Fill in release details here: +# Fill in release details here: RELEASE_URL = "http://people.apache.org/~pwendell/spark-1.0.0-rc1/" RELEASE_KEY = "9E4FE3AF" RELEASE_REPOSITORY = "https://repository.apache.org/content/repositories/orgapachespark-1006/" RELEASE_VERSION = "1.0.0" SCALA_VERSION = "2.10.4" SCALA_BINARY_VERSION = "2.10" -## +# LOG_FILE_NAME = "spark_audit_%s" % time.strftime("%h_%m_%Y_%I_%M_%S") LOG_FILE = open(LOG_FILE_NAME, 'w') -WORK_DIR = "/tmp/audit_%s" % int(time.time()) +WORK_DIR = "/tmp/audit_%s" % int(time.time()) MAVEN_CMD = "mvn" GPG_CMD = "gpg" @@ -50,54 +50,62 @@ # Track failures failures = [] + def clean_work_files(): - print "OK to delete scratch directory '%s'? (y/N): " % WORK_DIR - response = raw_input() - if response == "y": - shutil.rmtree(WORK_DIR) - print "Should I delete the log output file '%s'? (y/N): " % LOG_FILE_NAME - response = raw_input() - if response == "y": - os.unlink(LOG_FILE_NAME) + print "OK to delete scratch directory '%s'? (y/N): " % WORK_DIR + response = raw_input() + if response == "y": + shutil.rmtree(WORK_DIR) + print "Should I delete the log output file '%s'? (y/N): " % LOG_FILE_NAME + response = raw_input() + if response == "y": + os.unlink(LOG_FILE_NAME) + def run_cmd(cmd, exit_on_failure=True): - print >> LOG_FILE, "Running command: %s" % cmd - ret = subprocess.call(cmd, shell=True, stdout=LOG_FILE, stderr=LOG_FILE) - if ret != 0 and exit_on_failure: - print "Command failed: %s" % cmd - clean_work_files() - sys.exit(-1) - return ret + print >> LOG_FILE, "Running command: %s" % cmd + ret = subprocess.call(cmd, shell=True, stdout=LOG_FILE, stderr=LOG_FILE) + if ret != 0 and exit_on_failure: + print "Command failed: %s" % cmd + clean_work_files() + sys.exit(-1) + return ret + def run_cmd_with_output(cmd): - print >> sys.stderr, "Running command: %s" % cmd - return subprocess.check_output(cmd, shell=True, stderr=LOG_FILE) + print >> sys.stderr, "Running command: %s" % cmd + return subprocess.check_output(cmd, shell=True, stderr=LOG_FILE) + def test(bool, str): - if bool: - return passed(str) - failed(str) + if bool: + return passed(str) + failed(str) + def passed(str): - print "[PASSED] %s" % str + print "[PASSED] %s" % str + def failed(str): - failures.append(str) - print "[**FAILED**] %s" % str + failures.append(str) + print "[**FAILED**] %s" % str + def get_url(url): - return urllib2.urlopen(url).read() + return urllib2.urlopen(url).read() + original_dir = os.getcwd() -# For each of these modules, we'll test an 'empty' application in sbt and +# For each of these modules, we'll test an 'empty' application in sbt and # maven that links against them. This will catch issues with messed up # dependencies within those projects. modules = [ - "spark-core", "spark-bagel", "spark-mllib", "spark-streaming", "spark-repl", - "spark-graphx", "spark-streaming-flume", "spark-streaming-kafka", - "spark-streaming-mqtt", "spark-streaming-twitter", "spark-streaming-zeromq", - "spark-catalyst", "spark-sql", "spark-hive" + "spark-core", "spark-bagel", "spark-mllib", "spark-streaming", "spark-repl", + "spark-graphx", "spark-streaming-flume", "spark-streaming-kafka", + "spark-streaming-mqtt", "spark-streaming-twitter", "spark-streaming-zeromq", + "spark-catalyst", "spark-sql", "spark-hive" ] modules = map(lambda m: "%s_%s" % (m, SCALA_BINARY_VERSION), modules) @@ -106,54 +114,57 @@ def get_url(url): cache_ivy_spark = "~/.ivy2/cache/org.apache.spark" local_maven_kafka = "~/.m2/repository/org/apache/kafka" local_maven_kafka = "~/.m2/repository/org/apache/spark" + + def ensure_path_not_present(x): - if os.path.exists(os.path.expanduser(x)): - print "Please remove %s, it can interfere with testing published artifacts." % x - sys.exit(-1) + if os.path.exists(os.path.expanduser(x)): + print "Please remove %s, it can interfere with testing published artifacts." % x + sys.exit(-1) + map(ensure_path_not_present, [local_ivy_spark, cache_ivy_spark, local_maven_kafka]) -# SBT build tests +# SBT build tests os.chdir("blank_sbt_build") os.environ["SPARK_VERSION"] = RELEASE_VERSION os.environ["SCALA_VERSION"] = SCALA_VERSION os.environ["SPARK_RELEASE_REPOSITORY"] = RELEASE_REPOSITORY os.environ["SPARK_AUDIT_MASTER"] = "local" for module in modules: - os.environ["SPARK_MODULE"] = module - ret = run_cmd("sbt clean update", exit_on_failure=False) - test(ret == 0, "sbt build against '%s' module" % module) + os.environ["SPARK_MODULE"] = module + ret = run_cmd("sbt clean update", exit_on_failure=False) + test(ret == 0, "sbt build against '%s' module" % module) os.chdir(original_dir) # SBT application tests for app in ["sbt_app_core", "sbt_app_graphx", "sbt_app_streaming", "sbt_app_sql", "sbt_app_hive"]: - os.chdir(app) - ret = run_cmd("sbt clean run", exit_on_failure=False) - test(ret == 0, "sbt application (%s)" % app) - os.chdir(original_dir) + os.chdir(app) + ret = run_cmd("sbt clean run", exit_on_failure=False) + test(ret == 0, "sbt application (%s)" % app) + os.chdir(original_dir) # Maven build tests os.chdir("blank_maven_build") for module in modules: - cmd = ('%s --update-snapshots -Dspark.release.repository="%s" -Dspark.version="%s" ' - '-Dspark.module="%s" clean compile' % - (MAVEN_CMD, RELEASE_REPOSITORY, RELEASE_VERSION, module)) - ret = run_cmd(cmd, exit_on_failure=False) - test(ret == 0, "maven build against '%s' module" % module) + cmd = ('%s --update-snapshots -Dspark.release.repository="%s" -Dspark.version="%s" ' + '-Dspark.module="%s" clean compile' % + (MAVEN_CMD, RELEASE_REPOSITORY, RELEASE_VERSION, module)) + ret = run_cmd(cmd, exit_on_failure=False) + test(ret == 0, "maven build against '%s' module" % module) os.chdir(original_dir) os.chdir("maven_app_core") mvn_exec_cmd = ('%s --update-snapshots -Dspark.release.repository="%s" -Dspark.version="%s" ' '-Dscala.binary.version="%s" clean compile ' - 'exec:java -Dexec.mainClass="SimpleApp"' % - (MAVEN_CMD, RELEASE_REPOSITORY, RELEASE_VERSION, SCALA_BINARY_VERSION)) + 'exec:java -Dexec.mainClass="SimpleApp"' % + (MAVEN_CMD, RELEASE_REPOSITORY, RELEASE_VERSION, SCALA_BINARY_VERSION)) ret = run_cmd(mvn_exec_cmd, exit_on_failure=False) test(ret == 0, "maven application (core)") os.chdir(original_dir) # Binary artifact tests if os.path.exists(WORK_DIR): - print "Working directory '%s' already exists" % WORK_DIR - sys.exit(-1) + print "Working directory '%s' already exists" % WORK_DIR + sys.exit(-1) os.mkdir(WORK_DIR) os.chdir(WORK_DIR) @@ -162,66 +173,66 @@ def ensure_path_not_present(x): artifacts = r.findall(index_page) for artifact in artifacts: - print "==== Verifying download integrity for artifact: %s ====" % artifact - - artifact_url = "%s/%s" % (RELEASE_URL, artifact) - run_cmd("wget %s" % artifact_url) - - key_file = "%s.asc" % artifact - run_cmd("wget %s/%s" % (RELEASE_URL, key_file)) - - run_cmd("wget %s%s" % (artifact_url, ".sha")) - - # Verify signature - run_cmd("%s --keyserver pgp.mit.edu --recv-key %s" % (GPG_CMD, RELEASE_KEY)) - run_cmd("%s %s" % (GPG_CMD, key_file)) - passed("Artifact signature verified.") - - # Verify md5 - my_md5 = run_cmd_with_output("%s --print-md MD5 %s" % (GPG_CMD, artifact)).strip() - release_md5 = get_url("%s.md5" % artifact_url).strip() - test(my_md5 == release_md5, "Artifact MD5 verified.") - - # Verify sha - my_sha = run_cmd_with_output("%s --print-md SHA512 %s" % (GPG_CMD, artifact)).strip() - release_sha = get_url("%s.sha" % artifact_url).strip() - test(my_sha == release_sha, "Artifact SHA verified.") - - # Verify Apache required files - dir_name = artifact.replace(".tgz", "") - run_cmd("tar xvzf %s" % artifact) - base_files = os.listdir(dir_name) - test("CHANGES.txt" in base_files, "Tarball contains CHANGES.txt file") - test("NOTICE" in base_files, "Tarball contains NOTICE file") - test("LICENSE" in base_files, "Tarball contains LICENSE file") - - os.chdir(WORK_DIR) - + print "==== Verifying download integrity for artifact: %s ====" % artifact + + artifact_url = "%s/%s" % (RELEASE_URL, artifact) + run_cmd("wget %s" % artifact_url) + + key_file = "%s.asc" % artifact + run_cmd("wget %s/%s" % (RELEASE_URL, key_file)) + + run_cmd("wget %s%s" % (artifact_url, ".sha")) + + # Verify signature + run_cmd("%s --keyserver pgp.mit.edu --recv-key %s" % (GPG_CMD, RELEASE_KEY)) + run_cmd("%s %s" % (GPG_CMD, key_file)) + passed("Artifact signature verified.") + + # Verify md5 + my_md5 = run_cmd_with_output("%s --print-md MD5 %s" % (GPG_CMD, artifact)).strip() + release_md5 = get_url("%s.md5" % artifact_url).strip() + test(my_md5 == release_md5, "Artifact MD5 verified.") + + # Verify sha + my_sha = run_cmd_with_output("%s --print-md SHA512 %s" % (GPG_CMD, artifact)).strip() + release_sha = get_url("%s.sha" % artifact_url).strip() + test(my_sha == release_sha, "Artifact SHA verified.") + + # Verify Apache required files + dir_name = artifact.replace(".tgz", "") + run_cmd("tar xvzf %s" % artifact) + base_files = os.listdir(dir_name) + test("CHANGES.txt" in base_files, "Tarball contains CHANGES.txt file") + test("NOTICE" in base_files, "Tarball contains NOTICE file") + test("LICENSE" in base_files, "Tarball contains LICENSE file") + + os.chdir(WORK_DIR) + for artifact in artifacts: - print "==== Verifying build and tests for artifact: %s ====" % artifact - os.chdir(os.path.join(WORK_DIR, dir_name)) - - os.environ["MAVEN_OPTS"] = "-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" - # Verify build - print "==> Running build" - run_cmd("sbt assembly") - passed("sbt build successful") - run_cmd("%s package -DskipTests" % MAVEN_CMD) - passed("Maven build successful") - - # Verify tests - print "==> Performing unit tests" - run_cmd("%s test" % MAVEN_CMD) - passed("Tests successful") - os.chdir(WORK_DIR) + print "==== Verifying build and tests for artifact: %s ====" % artifact + os.chdir(os.path.join(WORK_DIR, dir_name)) + + os.environ["MAVEN_OPTS"] = "-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" + # Verify build + print "==> Running build" + run_cmd("sbt assembly") + passed("sbt build successful") + run_cmd("%s package -DskipTests" % MAVEN_CMD) + passed("Maven build successful") + + # Verify tests + print "==> Performing unit tests" + run_cmd("%s test" % MAVEN_CMD) + passed("Tests successful") + os.chdir(WORK_DIR) clean_work_files() if len(failures) == 0: - print "ALL TESTS PASSED" + print "ALL TESTS PASSED" else: - print "SOME TESTS DID NOT PASS" - for f in failures: - print f + print "SOME TESTS DID NOT PASS" + for f in failures: + print f os.chdir(original_dir) diff --git a/dev/create-release/generate-changelist.py b/dev/create-release/generate-changelist.py index 13b744ec1b37e..de1b5d4ae1314 100755 --- a/dev/create-release/generate-changelist.py +++ b/dev/create-release/generate-changelist.py @@ -29,16 +29,16 @@ import subprocess import time import traceback - + SPARK_HOME = os.environ["SPARK_HOME"] NEW_RELEASE_VERSION = "1.0.0" PREV_RELEASE_GIT_TAG = "v0.9.1" - -CHANGELIST = "CHANGES.txt" + +CHANGELIST = "CHANGES.txt" OLD_CHANGELIST = "%s.old" % (CHANGELIST) NEW_CHANGELIST = "%s.new" % (CHANGELIST) TMP_CHANGELIST = "%s.tmp" % (CHANGELIST) - + # date before first PR in TLP Spark repo SPARK_REPO_CHANGE_DATE1 = time.strptime("2014-02-26", "%Y-%m-%d") # date after last PR in incubator Spark repo @@ -46,99 +46,103 @@ # Threshold PR number that differentiates PRs to TLP # and incubator repos SPARK_REPO_PR_NUM_THRESH = 200 - + LOG_FILE_NAME = "changes_%s" % time.strftime("%h_%m_%Y_%I_%M_%S") LOG_FILE = open(LOG_FILE_NAME, 'w') - + + def run_cmd(cmd): - try: - print >> LOG_FILE, "Running command: %s" % cmd - output = subprocess.check_output(cmd, shell=True, stderr=LOG_FILE) - print >> LOG_FILE, "Output: %s" % output - return output - except: - traceback.print_exc() - cleanup() - sys.exit(1) - + try: + print >> LOG_FILE, "Running command: %s" % cmd + output = subprocess.check_output(cmd, shell=True, stderr=LOG_FILE) + print >> LOG_FILE, "Output: %s" % output + return output + except: + traceback.print_exc() + cleanup() + sys.exit(1) + + def append_to_changelist(string): - with open(TMP_CHANGELIST, "a") as f: - print >> f, string - -def cleanup(ask = True): - if ask == True: - print "OK to delete temporary and log files? (y/N): " - response = raw_input() - if ask == False or (ask == True and response == "y"): - if os.path.isfile(TMP_CHANGELIST): - os.remove(TMP_CHANGELIST) - if os.path.isfile(OLD_CHANGELIST): - os.remove(OLD_CHANGELIST) - LOG_FILE.close() - os.remove(LOG_FILE_NAME) - + with open(TMP_CHANGELIST, "a") as f: + print >> f, string + + +def cleanup(ask=True): + if ask is True: + print "OK to delete temporary and log files? (y/N): " + response = raw_input() + if ask is False or (ask is True and response == "y"): + if os.path.isfile(TMP_CHANGELIST): + os.remove(TMP_CHANGELIST) + if os.path.isfile(OLD_CHANGELIST): + os.remove(OLD_CHANGELIST) + LOG_FILE.close() + os.remove(LOG_FILE_NAME) + + print "Generating new %s for Spark release %s" % (CHANGELIST, NEW_RELEASE_VERSION) os.chdir(SPARK_HOME) if os.path.isfile(TMP_CHANGELIST): - os.remove(TMP_CHANGELIST) + os.remove(TMP_CHANGELIST) if os.path.isfile(OLD_CHANGELIST): - os.remove(OLD_CHANGELIST) - + os.remove(OLD_CHANGELIST) + append_to_changelist("Spark Change Log") append_to_changelist("----------------") append_to_changelist("") append_to_changelist("Release %s" % NEW_RELEASE_VERSION) append_to_changelist("") - + print "Getting commits between tag %s and HEAD" % PREV_RELEASE_GIT_TAG hashes = run_cmd("git log %s..HEAD --pretty='%%h'" % PREV_RELEASE_GIT_TAG).split() - + print "Getting details of %s commits" % len(hashes) for h in hashes: - date = run_cmd("git log %s -1 --pretty='%%ad' --date=iso | head -1" % h).strip() - subject = run_cmd("git log %s -1 --pretty='%%s' | head -1" % h).strip() - body = run_cmd("git log %s -1 --pretty='%%b'" % h) - committer = run_cmd("git log %s -1 --pretty='%%cn <%%ce>' | head -1" % h).strip() - body_lines = body.split("\n") - - if "Merge pull" in subject: - ## Parse old format commit message - append_to_changelist(" %s %s" % (h, date)) - append_to_changelist(" %s" % subject) - append_to_changelist(" [%s]" % body_lines[0]) - append_to_changelist("") - - elif "maven-release" not in subject: - ## Parse new format commit message - # Get authors from commit message, committer otherwise - authors = [committer] - if "Author:" in body: - authors = [line.split(":")[1].strip() for line in body_lines if "Author:" in line] - - # Generate GitHub PR URL for easy access if possible - github_url = "" - if "Closes #" in body: - pr_num = [line.split()[1].lstrip("#") for line in body_lines if "Closes #" in line][0] - github_url = "github.com/apache/spark/pull/%s" % pr_num - day = time.strptime(date.split()[0], "%Y-%m-%d") - if day < SPARK_REPO_CHANGE_DATE1 or (day < SPARK_REPO_CHANGE_DATE2 and pr_num < SPARK_REPO_PR_NUM_THRESH): - github_url = "github.com/apache/incubator-spark/pull/%s" % pr_num - - append_to_changelist(" %s" % subject) - append_to_changelist(" %s" % ', '.join(authors)) - # for author in authors: - # append_to_changelist(" %s" % author) - append_to_changelist(" %s" % date) - if len(github_url) > 0: - append_to_changelist(" Commit: %s, %s" % (h, github_url)) - else: - append_to_changelist(" Commit: %s" % h) - append_to_changelist("") - + date = run_cmd("git log %s -1 --pretty='%%ad' --date=iso | head -1" % h).strip() + subject = run_cmd("git log %s -1 --pretty='%%s' | head -1" % h).strip() + body = run_cmd("git log %s -1 --pretty='%%b'" % h) + committer = run_cmd("git log %s -1 --pretty='%%cn <%%ce>' | head -1" % h).strip() + body_lines = body.split("\n") + + if "Merge pull" in subject: + # Parse old format commit message + append_to_changelist(" %s %s" % (h, date)) + append_to_changelist(" %s" % subject) + append_to_changelist(" [%s]" % body_lines[0]) + append_to_changelist("") + + elif "maven-release" not in subject: + # Parse new format commit message + # Get authors from commit message, committer otherwise + authors = [committer] + if "Author:" in body: + authors = [line.split(":")[1].strip() for line in body_lines if "Author:" in line] + + # Generate GitHub PR URL for easy access if possible + github_url = "" + if "Closes #" in body: + pr_num = [line.split()[1].lstrip("#") for line in body_lines if "Closes #" in line][0] + github_url = "github.com/apache/spark/pull/%s" % pr_num + day = time.strptime(date.split()[0], "%Y-%m-%d") + if day < SPARK_REPO_CHANGE_DATE1 or + (day < SPARK_REPO_CHANGE_DATE2 and pr_num < SPARK_REPO_PR_NUM_THRESH): + github_url = "github.com/apache/incubator-spark/pull/%s" % pr_num + + append_to_changelist(" %s" % subject) + append_to_changelist(" %s" % ', '.join(authors)) + # for author in authors: + # append_to_changelist(" %s" % author) + append_to_changelist(" %s" % date) + if len(github_url) > 0: + append_to_changelist(" Commit: %s, %s" % (h, github_url)) + else: + append_to_changelist(" Commit: %s" % h) + append_to_changelist("") + # Append old change list -print "Appending changelist from tag %s" % PREV_RELEASE_GIT_TAG +print "Appending changelist from tag %s" % PREV_RELEASE_GIT_TAG run_cmd("git show %s:%s | tail -n +3 >> %s" % (PREV_RELEASE_GIT_TAG, CHANGELIST, TMP_CHANGELIST)) run_cmd("cp %s %s" % (TMP_CHANGELIST, NEW_CHANGELIST)) print "New change list generated as %s" % NEW_CHANGELIST cleanup(False) - diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 83618c8068d35..7f744d5589ef7 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -21,7 +21,7 @@ # usage: ./apache-pr-merge.py (see config env vars below) # # This utility assumes you already have local a Spark git folder and that you -# have added remotes corresponding to both (i) the github apache Spark +# have added remotes corresponding to both (i) the github apache Spark # mirror and (ii) the apache git repo. import json @@ -33,10 +33,10 @@ import urllib2 try: - import jira.client - JIRA_IMPORTED=True + import jira.client + JIRA_IMPORTED = True except ImportError: - JIRA_IMPORTED=False + JIRA_IMPORTED = False # Location of your Spark git development area SPARK_HOME = os.environ.get("SPARK_HOME", "/home/patrick/Documents/spark") @@ -58,204 +58,217 @@ os.chdir(SPARK_HOME) + def get_json(url): - try: - return json.load(urllib2.urlopen(url)) - except urllib2.HTTPError as e: - print "Unable to fetch URL, exiting: %s" % url - sys.exit(-1) + try: + return json.load(urllib2.urlopen(url)) + except urllib2.HTTPError as e: + print "Unable to fetch URL, exiting: %s" % url + sys.exit(-1) + def fail(msg): - print msg - clean_up() - sys.exit(-1) + print msg + clean_up() + sys.exit(-1) + def run_cmd(cmd): - if isinstance(cmd, list): - return subprocess.check_output(cmd) - else: - return subprocess.check_output(cmd.split(" ")) + if isinstance(cmd, list): + return subprocess.check_output(cmd) + else: + return subprocess.check_output(cmd.split(" ")) + def continue_maybe(prompt): - result = raw_input("\n%s (y/n): " % prompt) - if result.lower() != "y": - fail("Okay, exiting") + result = raw_input("\n%s (y/n): " % prompt) + if result.lower() != "y": + fail("Okay, exiting") + original_head = run_cmd("git rev-parse HEAD")[:8] + def clean_up(): - print "Restoring head pointer to %s" % original_head - run_cmd("git checkout %s" % original_head) + print "Restoring head pointer to %s" % original_head + run_cmd("git checkout %s" % original_head) + + branches = run_cmd("git branch").replace(" ", "").split("\n") - branches = run_cmd("git branch").replace(" ", "").split("\n") + for branch in filter(lambda x: x.startswith(BRANCH_PREFIX), branches): + print "Deleting local branch %s" % branch + run_cmd("git branch -D %s" % branch) - for branch in filter(lambda x: x.startswith(BRANCH_PREFIX), branches): - print "Deleting local branch %s" % branch - run_cmd("git branch -D %s" % branch) # merge the requested PR and return the merge hash def merge_pr(pr_num, target_ref): - pr_branch_name = "%s_MERGE_PR_%s" % (BRANCH_PREFIX, pr_num) - target_branch_name = "%s_MERGE_PR_%s_%s" % (BRANCH_PREFIX, pr_num, target_ref.upper()) - run_cmd("git fetch %s pull/%s/head:%s" % (PR_REMOTE_NAME, pr_num, pr_branch_name)) - run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, target_ref, target_branch_name)) - run_cmd("git checkout %s" % target_branch_name) - - had_conflicts = False - try: - run_cmd(['git', 'merge', pr_branch_name, '--squash']) - except Exception as e: - msg = "Error merging: %s\nWould you like to manually fix-up this merge?" % e - continue_maybe(msg) - msg = "Okay, please fix any conflicts and 'git add' conflicting files... Finished?" - continue_maybe(msg) - had_conflicts = True - - commit_authors = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, - '--pretty=format:%an <%ae>']).split("\n") - distinct_authors = sorted(set(commit_authors), key=lambda x: commit_authors.count(x), - reverse=True) - primary_author = distinct_authors[0] - commits = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, - '--pretty=format:%h [%an] %s']).split("\n\n") - - merge_message_flags = [] - - for p in [title, body]: - merge_message_flags += ["-m", p] - - authors = "\n".join(["Author: %s" % a for a in distinct_authors]) - - merge_message_flags += ["-m", authors] + pr_branch_name = "%s_MERGE_PR_%s" % (BRANCH_PREFIX, pr_num) + target_branch_name = "%s_MERGE_PR_%s_%s" % (BRANCH_PREFIX, pr_num, target_ref.upper()) + run_cmd("git fetch %s pull/%s/head:%s" % (PR_REMOTE_NAME, pr_num, pr_branch_name)) + run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, target_ref, target_branch_name)) + run_cmd("git checkout %s" % target_branch_name) + + had_conflicts = False + try: + run_cmd(['git', 'merge', pr_branch_name, '--squash']) + except Exception as e: + msg = "Error merging: %s\nWould you like to manually fix-up this merge?" % e + continue_maybe(msg) + msg = "Okay, please fix any conflicts and 'git add' conflicting files... Finished?" + continue_maybe(msg) + had_conflicts = True + + commit_authors = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, + '--pretty=format:%an <%ae>']).split("\n") + distinct_authors = sorted(set(commit_authors), + key=lambda x: commit_authors.count(x), reverse=True) + primary_author = distinct_authors[0] + commits = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, + '--pretty=format:%h [%an] %s']).split("\n\n") + + merge_message_flags = [] + + for p in [title, body]: + merge_message_flags += ["-m", p] + + authors = "\n".join(["Author: %s" % a for a in distinct_authors]) + + merge_message_flags += ["-m", authors] + + if had_conflicts: + committer_name = run_cmd("git config --get user.name").strip() + committer_email = run_cmd("git config --get user.email").strip() + message = "This patch had conflicts when merged, resolved by\nCommitter: %s <%s>" % ( + committer_name, committer_email) + merge_message_flags += ["-m", message] + + # The string "Closes #%s" string is required for GitHub to correctly close the PR + merge_message_flags += [ + "-m", + "Closes #%s from %s and squashes the following commits:" % (pr_num, pr_repo_desc)] + for c in commits: + merge_message_flags += ["-m", c] + + run_cmd(['git', 'commit', '--author="%s"' % primary_author] + merge_message_flags) + + continue_maybe("Merge complete (local ref %s). Push to %s?" % ( + target_branch_name, PUSH_REMOTE_NAME)) + + try: + run_cmd('git push %s %s:%s' % (PUSH_REMOTE_NAME, target_branch_name, target_ref)) + except Exception as e: + clean_up() + fail("Exception while pushing: %s" % e) + + merge_hash = run_cmd("git rev-parse %s" % target_branch_name)[:8] + clean_up() + print("Pull request #%s merged!" % pr_num) + print("Merge hash: %s" % merge_hash) + return merge_hash - if had_conflicts: - committer_name = run_cmd("git config --get user.name").strip() - committer_email = run_cmd("git config --get user.email").strip() - message = "This patch had conflicts when merged, resolved by\nCommitter: %s <%s>" % ( - committer_name, committer_email) - merge_message_flags += ["-m", message] - # The string "Closes #%s" string is required for GitHub to correctly close the PR - merge_message_flags += ["-m", - "Closes #%s from %s and squashes the following commits:" % (pr_num, pr_repo_desc)] - for c in commits: - merge_message_flags += ["-m", c] +def cherry_pick(pr_num, merge_hash, default_branch): + pick_ref = raw_input("Enter a branch name [%s]: " % default_branch) + if pick_ref == "": + pick_ref = default_branch - run_cmd(['git', 'commit', '--author="%s"' % primary_author] + merge_message_flags) + pick_branch_name = "%s_PICK_PR_%s_%s" % (BRANCH_PREFIX, pr_num, pick_ref.upper()) - continue_maybe("Merge complete (local ref %s). Push to %s?" % ( - target_branch_name, PUSH_REMOTE_NAME)) + run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, pick_ref, pick_branch_name)) + run_cmd("git checkout %s" % pick_branch_name) + run_cmd("git cherry-pick -sx %s" % merge_hash) - try: - run_cmd('git push %s %s:%s' % (PUSH_REMOTE_NAME, target_branch_name, target_ref)) - except Exception as e: - clean_up() - fail("Exception while pushing: %s" % e) - - merge_hash = run_cmd("git rev-parse %s" % target_branch_name)[:8] - clean_up() - print("Pull request #%s merged!" % pr_num) - print("Merge hash: %s" % merge_hash) - return merge_hash + continue_maybe("Pick complete (local ref %s). Push to %s?" % ( + pick_branch_name, PUSH_REMOTE_NAME)) + try: + run_cmd('git push %s %s:%s' % (PUSH_REMOTE_NAME, pick_branch_name, pick_ref)) + except Exception as e: + clean_up() + fail("Exception while pushing: %s" % e) -def cherry_pick(pr_num, merge_hash, default_branch): - pick_ref = raw_input("Enter a branch name [%s]: " % default_branch) - if pick_ref == "": - pick_ref = default_branch - - pick_branch_name = "%s_PICK_PR_%s_%s" % (BRANCH_PREFIX, pr_num, pick_ref.upper()) - - run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, pick_ref, pick_branch_name)) - run_cmd("git checkout %s" % pick_branch_name) - run_cmd("git cherry-pick -sx %s" % merge_hash) - - continue_maybe("Pick complete (local ref %s). Push to %s?" % ( - pick_branch_name, PUSH_REMOTE_NAME)) - - try: - run_cmd('git push %s %s:%s' % (PUSH_REMOTE_NAME, pick_branch_name, pick_ref)) - except Exception as e: + pick_hash = run_cmd("git rev-parse %s" % pick_branch_name)[:8] clean_up() - fail("Exception while pushing: %s" % e) - pick_hash = run_cmd("git rev-parse %s" % pick_branch_name)[:8] - clean_up() + print("Pull request #%s picked into %s!" % (pr_num, pick_ref)) + print("Pick hash: %s" % pick_hash) + return pick_ref - print("Pull request #%s picked into %s!" % (pr_num, pick_ref)) - print("Pick hash: %s" % pick_hash) - return pick_ref def fix_version_from_branch(branch, versions): - # Note: Assumes this is a sorted (newest->oldest) list of un-released versions - if branch == "master": - return versions[0] - else: - branch_ver = branch.replace("branch-", "") - return filter(lambda x: x.name.startswith(branch_ver), versions)[-1] + # Note: Assumes this is a sorted (newest->oldest) list of un-released versions + if branch == "master": + return versions[0] + else: + branch_ver = branch.replace("branch-", "") + return filter(lambda x: x.name.startswith(branch_ver), versions)[-1] + def resolve_jira(title, merge_branches, comment): - asf_jira = jira.client.JIRA({'server': JIRA_API_BASE}, - basic_auth=(JIRA_USERNAME, JIRA_PASSWORD)) - - default_jira_id = "" - search = re.findall("SPARK-[0-9]{4,5}", title) - if len(search) > 0: - default_jira_id = search[0] - - jira_id = raw_input("Enter a JIRA id [%s]: " % default_jira_id) - if jira_id == "": - jira_id = default_jira_id - - try: - issue = asf_jira.issue(jira_id) - except Exception as e: - fail("ASF JIRA could not find %s\n%s" % (jira_id, e)) - - cur_status = issue.fields.status.name - cur_summary = issue.fields.summary - cur_assignee = issue.fields.assignee - if cur_assignee == None: - cur_assignee = "NOT ASSIGNED!!!" - else: - cur_assignee = cur_assignee.displayName - - if cur_status == "Resolved" or cur_status == "Closed": - fail("JIRA issue %s already has status '%s'" % (jira_id, cur_status)) - print ("=== JIRA %s ===" % jira_id) - print ("summary\t\t%s\nassignee\t%s\nstatus\t\t%s\nurl\t\t%s/%s\n" % ( - cur_summary, cur_assignee, cur_status, JIRA_BASE, jira_id)) - - versions = asf_jira.project_versions("SPARK") - versions = sorted(versions, key = lambda x: x.name, reverse=True) - versions = filter(lambda x: x.raw['released'] == False, versions) - - default_fix_versions = map(lambda x: fix_version_from_branch(x, versions).name, merge_branches) - for v in default_fix_versions: - # Handles the case where we have forked a release branch but not yet made the release. - # In this case, if the PR is committed to the master branch and the release branch, we - # only consider the release branch to be the fix version. E.g. it is not valid to have - # both 1.1.0 and 1.0.0 as fix versions. - (major, minor, patch) = v.split(".") - if patch == "0": - previous = "%s.%s.%s" % (major, int(minor) - 1, 0) - if previous in default_fix_versions: - default_fix_versions = filter(lambda x: x != v, default_fix_versions) - default_fix_versions = ",".join(default_fix_versions) - - fix_versions = raw_input("Enter comma-separated fix version(s) [%s]: " % default_fix_versions) - if fix_versions == "": - fix_versions = default_fix_versions - fix_versions = fix_versions.replace(" ", "").split(",") - - def get_version_json(version_str): - return filter(lambda v: v.name == version_str, versions)[0].raw - jira_fix_versions = map(lambda v: get_version_json(v), fix_versions) - - resolve = filter(lambda a: a['name'] == "Resolve Issue", asf_jira.transitions(jira_id))[0] - asf_jira.transition_issue(jira_id, resolve["id"], fixVersions=jira_fix_versions, comment=comment) - - print "Succesfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions) + asf_jira = jira.client.JIRA({'server': JIRA_API_BASE}, + basic_auth=(JIRA_USERNAME, JIRA_PASSWORD)) + + default_jira_id = "" + search = re.findall("SPARK-[0-9]{4,5}", title) + if len(search) > 0: + default_jira_id = search[0] + + jira_id = raw_input("Enter a JIRA id [%s]: " % default_jira_id) + if jira_id == "": + jira_id = default_jira_id + + try: + issue = asf_jira.issue(jira_id) + except Exception as e: + fail("ASF JIRA could not find %s\n%s" % (jira_id, e)) + + cur_status = issue.fields.status.name + cur_summary = issue.fields.summary + cur_assignee = issue.fields.assignee + if cur_assignee is None: + cur_assignee = "NOT ASSIGNED!!!" + else: + cur_assignee = cur_assignee.displayName + + if cur_status == "Resolved" or cur_status == "Closed": + fail("JIRA issue %s already has status '%s'" % (jira_id, cur_status)) + print ("=== JIRA %s ===" % jira_id) + print ("summary\t\t%s\nassignee\t%s\nstatus\t\t%s\nurl\t\t%s/%s\n" % ( + cur_summary, cur_assignee, cur_status, JIRA_BASE, jira_id)) + + versions = asf_jira.project_versions("SPARK") + versions = sorted(versions, key=lambda x: x.name, reverse=True) + versions = filter(lambda x: x.raw['released'] is False, versions) + + default_fix_versions = map(lambda x: fix_version_from_branch(x, versions).name, merge_branches) + for v in default_fix_versions: + # Handles the case where we have forked a release branch but not yet made the release. + # In this case, if the PR is committed to the master branch and the release branch, we + # only consider the release branch to be the fix version. E.g. it is not valid to have + # both 1.1.0 and 1.0.0 as fix versions. + (major, minor, patch) = v.split(".") + if patch == "0": + previous = "%s.%s.%s" % (major, int(minor) - 1, 0) + if previous in default_fix_versions: + default_fix_versions = filter(lambda x: x != v, default_fix_versions) + default_fix_versions = ",".join(default_fix_versions) + + fix_versions = raw_input("Enter comma-separated fix version(s) [%s]: " % default_fix_versions) + if fix_versions == "": + fix_versions = default_fix_versions + fix_versions = fix_versions.replace(" ", "").split(",") + + def get_version_json(version_str): + return filter(lambda v: v.name == version_str, versions)[0].raw + + jira_fix_versions = map(lambda v: get_version_json(v), fix_versions) + + resolve = filter(lambda a: a['name'] == "Resolve Issue", asf_jira.transitions(jira_id))[0] + asf_jira.transition_issue( + jira_id, resolve["id"], fixVersions=jira_fix_versions, comment=comment) + + print "Succesfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions) + branches = get_json("%s/branches" % GITHUB_API_BASE) branch_names = filter(lambda x: x.startswith("branch-"), [x['name'] for x in branches]) @@ -273,28 +286,29 @@ def get_version_json(version_str): base_ref = pr["head"]["ref"] pr_repo_desc = "%s/%s" % (user_login, base_ref) -if pr["merged"] == True: - print "Pull request %s has already been merged, assuming you want to backport" % pr_num - merge_commit_desc = run_cmd(['git', 'log', '--merges', '--first-parent', - '--grep=pull request #%s' % pr_num, '--oneline']).split("\n")[0] - if merge_commit_desc == "": - fail("Couldn't find any merge commit for #%s, you may need to update HEAD." % pr_num) +if pr["merged"] is True: + print "Pull request %s has already been merged, assuming you want to backport" % pr_num + merge_commit_desc = run_cmd([ + 'git', 'log', '--merges', '--first-parent', + '--grep=pull request #%s' % pr_num, '--oneline']).split("\n")[0] + if merge_commit_desc == "": + fail("Couldn't find any merge commit for #%s, you may need to update HEAD." % pr_num) + + merge_hash = merge_commit_desc[:7] + message = merge_commit_desc[8:] - merge_hash = merge_commit_desc[:7] - message = merge_commit_desc[8:] - - print "Found: %s" % message - maybe_cherry_pick(pr_num, merge_hash, latest_branch) - sys.exit(0) + print "Found: %s" % message + maybe_cherry_pick(pr_num, merge_hash, latest_branch) + sys.exit(0) if not bool(pr["mergeable"]): - msg = "Pull request %s is not mergeable in its current form.\n" % pr_num + \ - "Continue? (experts only!)" - continue_maybe(msg) + msg = "Pull request %s is not mergeable in its current form.\n" % pr_num + \ + "Continue? (experts only!)" + continue_maybe(msg) print ("\n=== Pull Request #%s ===" % pr_num) print ("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( - title, pr_repo_desc, target_ref, url)) + title, pr_repo_desc, target_ref, url)) continue_maybe("Proceed with merging pull request #%s?" % pr_num) merged_refs = [target_ref] @@ -303,12 +317,12 @@ def get_version_json(version_str): pick_prompt = "Would you like to pick %s into another branch?" % merge_hash while raw_input("\n%s (y/n): " % pick_prompt).lower() == "y": - merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)] + merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)] if JIRA_IMPORTED: - continue_maybe("Would you like to update an associated JIRA?") - jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) - resolve_jira(title, merged_refs, jira_comment) + continue_maybe("Would you like to update an associated JIRA?") + jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) + resolve_jira(title, merged_refs, jira_comment) else: - print "Could not find jira-python library. Run 'sudo pip install jira-python' to install." - print "Exiting without trying to close the associated JIRA." + print "Could not find jira-python library. Run 'sudo pip install jira-python' to install." + print "Exiting without trying to close the associated JIRA." From 90e281b55aecbfbe4431ac582311d5790fe7aad3 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 26 May 2014 22:05:23 -0700 Subject: [PATCH 435/641] SPARK-1933: Throw a more meaningful exception when a directory is passed to addJar/addFile. https://issues.apache.org/jira/browse/SPARK-1933 Author: Reynold Xin Closes #888 from rxin/addfile and squashes the following commits: 8c402a3 [Reynold Xin] Updated comment. ff6c162 [Reynold Xin] SPARK-1933: Throw a more meaningful exception when a directory is passed to addJar/addFile. --- core/src/main/scala/org/apache/spark/HttpFileServer.scala | 7 +++++++ core/src/main/scala/org/apache/spark/SparkContext.scala | 5 ++--- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala index a6e300d345786..0e3750fdde415 100644 --- a/core/src/main/scala/org/apache/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -59,6 +59,13 @@ private[spark] class HttpFileServer(securityManager: SecurityManager) extends Lo } def addFileToDir(file: File, dir: File) : String = { + // Check whether the file is a directory. If it is, throw a more meaningful exception. + // If we don't catch this, Guava throws a very confusing error message: + // java.io.FileNotFoundException: [file] (No such file or directory) + // even though the directory ([file]) exists. + if (file.isDirectory) { + throw new IllegalArgumentException(s"$file cannot be a directory.") + } Files.copy(file, new File(dir, file.getName)) dir + "/" + file.getName } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 49737fa4be56b..03ceff8bf1fb0 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -794,7 +794,7 @@ class SparkContext(config: SparkConf) extends Logging { addedFiles(key) = System.currentTimeMillis // Fetch the file locally in case a job is executed using DAGScheduler.runLocally(). - Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf, env.securityManager) + Utils.fetchFile(path, new File(SparkFiles.getRootDirectory()), conf, env.securityManager) logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) postEnvironmentUpdate() @@ -932,13 +932,12 @@ class SparkContext(config: SparkConf) extends Logging { try { env.httpFileServer.addJar(new File(fileName)) } catch { - case e: Exception => { + case e: Exception => // For now just log an error but allow to go through so spark examples work. // The spark examples don't really need the jar distributed since its also // the app jar. logError("Error adding jar (" + e + "), was the --addJars option used?") null - } } } else { env.httpFileServer.addJar(new File(uri.getPath)) From 549830b0db2c8b069391224f3a73bb0d7f397f71 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 26 May 2014 23:17:39 -0700 Subject: [PATCH 436/641] SPARK-1932: Fix race conditions in onReceiveCallback and cachedPeers `var cachedPeers: Seq[BlockManagerId] = null` is used in `def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel)` without proper protection. There are two place will call `replicate(blockId, bytesAfterPut, level)` * https://github.com/apache/spark/blob/17f3075bc4aa8cbed165f7b367f70e84b1bc8db9/core/src/main/scala/org/apache/spark/storage/BlockManager.scala#L644 runs in `connectionManager.futureExecContext` * https://github.com/apache/spark/blob/17f3075bc4aa8cbed165f7b367f70e84b1bc8db9/core/src/main/scala/org/apache/spark/storage/BlockManager.scala#L752 `doPut` runs in `connectionManager.handleMessageExecutor`. `org.apache.spark.storage.BlockManagerWorker` calls `blockManager.putBytes` in `connectionManager.handleMessageExecutor`. As they run in different `Executor`s, this is a race condition which may cause the memory pointed by `cachedPeers` is not correct even if `cachedPeers != null`. The race condition of `onReceiveCallback` is that it's set in `BlockManagerWorker` but read in a different thread in `ConnectionManager.handleMessageExecutor`. Author: zsxwing Closes #887 from zsxwing/SPARK-1932 and squashes the following commits: 524f69c [zsxwing] SPARK-1932: Fix race conditions in onReceiveCallback and cachedPeers --- .../scala/org/apache/spark/network/ConnectionManager.scala | 3 ++- .../src/main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index dcbbc1853186b..5dd5fd0047c0d 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -93,7 +93,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, implicit val futureExecContext = ExecutionContext.fromExecutor( Utils.newDaemonCachedThreadPool("Connection manager future execution context")) - private var onReceiveCallback: (BufferMessage, ConnectionManagerId) => Option[Message]= null + @volatile + private var onReceiveCallback: (BufferMessage, ConnectionManagerId) => Option[Message] = null private val authEnabled = securityManager.isAuthenticationEnabled() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 6534095811907..6e450081dcb11 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -772,7 +772,7 @@ private[spark] class BlockManager( /** * Replicate block to another node. */ - var cachedPeers: Seq[BlockManagerId] = null + @volatile var cachedPeers: Seq[BlockManagerId] = null private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel) { val tLevel = StorageLevel( level.useDisk, level.useMemory, level.useOffHeap, level.deserialized, 1) From 95e4c9c6fb153b7f0aa4c442c4bdb6552d326640 Mon Sep 17 00:00:00 2001 From: lianhuiwang Date: Tue, 27 May 2014 11:53:38 -0700 Subject: [PATCH 437/641] bugfix worker DriverStateChanged state should match DriverState.FAILED bugfix worker DriverStateChanged state should match DriverState.FAILED Author: lianhuiwang Closes #864 from lianhuiwang/master and squashes the following commits: 480ce94 [lianhuiwang] address aarondav comments f2b5970 [lianhuiwang] bugfix worker DriverStateChanged state should match DriverState.FAILED --- .../main/scala/org/apache/spark/deploy/worker/Worker.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 8b6747977eb87..100de26170a50 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -317,10 +317,14 @@ private[spark] class Worker( state match { case DriverState.ERROR => logWarning(s"Driver $driverId failed with unrecoverable exception: ${exception.get}") + case DriverState.FAILED => + logWarning(s"Driver $driverId exited with failure") case DriverState.FINISHED => logInfo(s"Driver $driverId exited successfully") case DriverState.KILLED => logInfo(s"Driver $driverId was killed by user") + case _ => + logDebug(s"Driver $driverId changed state to $state") } masterLock.synchronized { master ! DriverStateChanged(driverId, state, exception) From d1375a2bff846f2c4274e14545924646852895f9 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 27 May 2014 14:53:57 -0700 Subject: [PATCH 438/641] [SPARK-1926] [SQL] Nullability of Max/Min/First should be true. Nullability of `Max`/`Min`/`First` should be `true` because they return `null` if there are no rows. Author: Takuya UESHIN Closes #881 from ueshin/issues/SPARK-1926 and squashes the following commits: 322610f [Takuya UESHIN] Fix nullability of Min/Max/First. --- .../apache/spark/sql/catalyst/expressions/aggregates.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 79937b129aeae..b49a4614eacab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -90,7 +90,7 @@ abstract class AggregateFunction case class Min(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { override def references = child.references - override def nullable = child.nullable + override def nullable = true override def dataType = child.dataType override def toString = s"MIN($child)" @@ -120,7 +120,7 @@ case class MinFunction(expr: Expression, base: AggregateExpression) extends Aggr case class Max(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { override def references = child.references - override def nullable = child.nullable + override def nullable = true override def dataType = child.dataType override def toString = s"MAX($child)" @@ -257,7 +257,7 @@ case class SumDistinct(child: Expression) case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { override def references = child.references - override def nullable = child.nullable + override def nullable = true override def dataType = child.dataType override def toString = s"FIRST($child)" From 3b0babad1f0856ee16f9d58e1ead30779a4a6310 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 27 May 2014 14:55:23 -0700 Subject: [PATCH 439/641] [SPARK-1915] [SQL] AverageFunction should not count if the evaluated value is null. Average values are difference between the calculation is done partially or not partially. Because `AverageFunction` (in not-partially calculation) counts even if the evaluated value is null. Author: Takuya UESHIN Closes #862 from ueshin/issues/SPARK-1915 and squashes the following commits: b1ff3c0 [Takuya UESHIN] Modify AverageFunction not to count if the evaluated value is null. --- .../spark/sql/catalyst/expressions/aggregates.scala | 9 ++++++--- .../scala/org/apache/spark/sql/DslQuerySuite.scala | 10 ++++++++++ 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index b49a4614eacab..c902433688943 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -281,14 +281,17 @@ case class AverageFunction(expr: Expression, base: AggregateExpression) private val sum = MutableLiteral(zero.eval(EmptyRow)) private val sumAsDouble = Cast(sum, DoubleType) - private val addFunction = Add(sum, Coalesce(Seq(expr, zero))) + private def addFunction(value: Any) = Add(sum, Literal(value)) override def eval(input: Row): Any = sumAsDouble.eval(EmptyRow).asInstanceOf[Double] / count.toDouble override def update(input: Row): Unit = { - count += 1 - sum.update(addFunction, input) + val evaluatedExpr = expr.eval(input) + if (evaluatedExpr != null) { + count += 1 + sum.update(addFunction(evaluatedExpr), input) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 8197e8a18d447..fb599e1e01e73 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -115,6 +115,16 @@ class DslQuerySuite extends QueryTest { 2.0) } + test("null average") { + checkAnswer( + testData3.groupBy()(Average('b)), + 2.0) + + checkAnswer( + testData3.groupBy()(Average('b), CountDistinct('b :: Nil)), + (2.0, 1) :: Nil) + } + test("count") { assert(testData2.count() === testData2.map(_ => 1).count()) } From 068256745052b0aa947dd8c16b1f1d73d8e4631e Mon Sep 17 00:00:00 2001 From: LY Lai Date: Tue, 27 May 2014 16:08:38 -0700 Subject: [PATCH 440/641] [SQL] SPARK-1922 Allow underscore in column name of a struct field https://issues.apache.org/jira/browse/SPARK-1922 . Author: LY Lai Closes #873 from lyuanlai/master and squashes the following commits: 2253263 [LY Lai] Allow underscore in struct field column name --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../sql/hive/HiveMetastoreCatalogSuite.scala | 32 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 636c4f7b93190..9f74e0334f727 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -201,7 +201,7 @@ object HiveMetastoreTypes extends RegexParsers { } protected lazy val structField: Parser[StructField] = - "[a-zA-Z0-9]*".r ~ ":" ~ dataType ^^ { + "[a-zA-Z0-9_]*".r ~ ":" ~ dataType ^^ { case name ~ _ ~ tpe => StructField(name, tpe, nullable = true) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala new file mode 100644 index 0000000000000..4a64b5f5eb1b4 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.types.{DataType, StructType} + +class HiveMetastoreCatalogSuite extends FunSuite { + + test("struct field should accept underscore in sub-column name") { + val metastr = "struct" + + val datatype = HiveMetastoreTypes.toDataType(metastr) + assert(datatype.isInstanceOf[StructType]) + } +} From 9df86835b60ce587c8b9bd4ad7410eebf59a179d Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 27 May 2014 22:17:50 -0700 Subject: [PATCH 441/641] [SPARK-1938] [SQL] ApproxCountDistinctMergeFunction should return Int value. `ApproxCountDistinctMergeFunction` should return `Int` value because the `dataType` of `ApproxCountDistinct` is `IntegerType`. Author: Takuya UESHIN Closes #893 from ueshin/issues/SPARK-1938 and squashes the following commits: 3970e88 [Takuya UESHIN] Remove a superfluous line. 5ad7ec1 [Takuya UESHIN] Make dataType for each of CountDistinct, ApproxCountDistinctMerge and ApproxCountDistinct LongType. cbe7c71 [Takuya UESHIN] Revert a change. fc3ac0f [Takuya UESHIN] Fix evaluated value type of ApproxCountDistinctMergeFunction to Int. --- .../spark/sql/catalyst/expressions/aggregates.scala | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index c902433688943..01947273b6ccc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -82,7 +82,6 @@ abstract class AggregateFunction override def dataType = base.dataType def update(input: Row): Unit - override def eval(input: Row): Any // Do we really need this? override def newInstance() = makeCopy(productIterator.map { case a: AnyRef => a }.toArray) @@ -166,7 +165,7 @@ case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpressi override def children = expressions override def references = expressions.flatMap(_.references).toSet override def nullable = false - override def dataType = IntegerType + override def dataType = LongType override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")})" override def newInstance() = new CountDistinctFunction(expressions, this) } @@ -184,7 +183,7 @@ case class ApproxCountDistinctMerge(child: Expression, relativeSD: Double) extends AggregateExpression with trees.UnaryNode[Expression] { override def references = child.references override def nullable = false - override def dataType = IntegerType + override def dataType = LongType override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" override def newInstance() = new ApproxCountDistinctMergeFunction(child, this, relativeSD) } @@ -193,7 +192,7 @@ case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05) extends PartialAggregate with trees.UnaryNode[Expression] { override def references = child.references override def nullable = false - override def dataType = IntegerType + override def dataType = LongType override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" override def asPartial: SplitEvaluation = { @@ -394,7 +393,7 @@ case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpressio } } - override def eval(input: Row): Any = seen.size + override def eval(input: Row): Any = seen.size.toLong } case class FirstFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { From 82eadc3b07d3f00eebd30811f981016e68cf60bf Mon Sep 17 00:00:00 2001 From: jmu Date: Tue, 27 May 2014 22:41:47 -0700 Subject: [PATCH 442/641] Fix doc about NetworkWordCount/JavaNetworkWordCount usage of spark streaming Usage: NetworkWordCount --> Usage: NetworkWordCount Usage: JavaNetworkWordCount --> Usage: JavaNetworkWordCount Author: jmu Closes #826 from jmu/master and squashes the following commits: 9fb7980 [jmu] Merge branch 'master' of https://github.com/jmu/spark b9a6b02 [jmu] Fix doc for NetworkWordCount/JavaNetworkWordCount Usage: NetworkWordCount --> Usage: NetworkWordCount --- docs/streaming-programming-guide.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 0c125eb693a8e..972b660262d14 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -234,12 +234,12 @@ Then, in a different terminal, you can start the example by using
      {% highlight bash %} -$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount local[2] localhost 9999 +$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount localhost 9999 {% endhighlight %}
      {% highlight bash %} -$ ./bin/run-example org.apache.spark.examples.streaming.JavaNetworkWordCount local[2] localhost 9999 +$ ./bin/run-example org.apache.spark.examples.streaming.JavaNetworkWordCount localhost 9999 {% endhighlight %}
      @@ -268,7 +268,7 @@ hello world {% highlight bash %} # TERMINAL 2: RUNNING NetworkWordCount or JavaNetworkWordCount -$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount local[2] localhost 9999 +$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount localhost 9999 ... ------------------------------------------- Time: 1357008430000 ms From 7801d44fd3bcf4d82e6db12574cc42bef15bf0e1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 28 May 2014 15:49:54 -0700 Subject: [PATCH 443/641] Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes --- docs/configuration.md | 815 ++++++++++++++++++--------------------- docs/quick-start.md | 8 +- docs/spark-standalone.md | 167 +++++++- 3 files changed, 554 insertions(+), 436 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index e5d955f23fe32..b6e7fd34eae68 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -2,19 +2,25 @@ layout: global title: Spark Configuration --- - * This will become a table of contents (this text will be scraped). {:toc} -Spark provides several locations to configure the system: +Spark provides three locations to configure the system: + +* [Spark properties](#spark-properties) control most application parameters and can be set by passing + a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) object to SparkContext, or through Java + system properties. +* [Environment variables](#environment-variables) can be used to set per-machine settings, such as + the IP address, through the `conf/spark-env.sh` script on each node. +* [Logging](#configuring-logging) can be configured through `log4j.properties`. # Spark Properties Spark properties control most application settings and are configured separately for each -application. The preferred way is to set them through -[SparkConf](api/scala/index.html#org.apache.spark.SparkConf) and passing it as an argument to your -SparkContext. SparkConf allows you to configure most of the common properties to initialize a -cluster (e.g. master URL and application name), as well as arbitrary key-value pairs through the +application. These properties can be set directly on a +[SparkConf](api/scala/index.html#org.apache.spark.SparkConf) and passed as an argument to your +SparkContext. SparkConf allows you to configure some of the common properties +(e.g. master URL and application name), as well as arbitrary key-value pairs through the `set()` method. For example, we could initialize an application as follows: {% highlight scala %} @@ -25,22 +31,37 @@ val conf = new SparkConf() val sc = new SparkContext(conf) {% endhighlight %} -## Loading Default Configurations +## Dynamically Loading Spark Properties +In some cases, you may want to avoid hard-coding certain configurations in a `SparkConf`. For +instance, if you'd like to run the same application with different masters or different +amounts of memory. Spark allows you to simply create an empty conf: -In the case of `spark-shell`, a SparkContext has already been created for you, so you cannot control -the configuration properties through SparkConf. However, you can still set configuration properties -through a default configuration file. By default, `spark-shell` (and more generally `spark-submit`) -will read configuration options from `conf/spark-defaults.conf`, in which each line consists of a -key and a value separated by whitespace. For example, +{% highlight scala %} +val sc = new SparkContext(new SparkConf()) +{% endhighlight %} + +Then, you can supply configuration values at runtime: +{% highlight bash %} +./bin/spark-submit --name "My fancy app" --master local[4] myApp.jar +{% endhighlight %} + +The Spark shell and [`spark-submit`](cluster-overview.html#launching-applications-with-spark-submit) +tool support two ways to load configurations dynamically. The first are command line options, +such as `--master`, as shown above. Running `./bin/spark-submit --help` will show the entire list +of options. + +`bin/spark-submit` will also read configuration options from `conf/spark-defaults.conf`, in which +each line consists of a key and a value separated by whitespace. For example: spark.master spark://5.6.7.8:7077 spark.executor.memory 512m spark.eventLog.enabled true spark.serializer org.apache.spark.serializer.KryoSerializer -Any values specified in the file will be passed on to the application, and merged with those -specified through SparkConf. If the same configuration property exists in both `spark-defaults.conf` -and SparkConf, then the latter will take precedence as it is the most application-specific. +Any values specified as flags or in the properties file will be passed on to the application +and merged with those specified through SparkConf. Properties set directly on the SparkConf +take highest precedence, then flags passed to `spark-submit` or `spark-shell`, then options +in the `spark-defaults.conf` file. ## Viewing Spark Properties @@ -49,19 +70,34 @@ This is a useful place to check to make sure that your properties have been set that only values explicitly specified through either `spark-defaults.conf` or SparkConf will appear. For all other configuration properties, you can assume the default value is used. -## All Configuration Properties +## Available Properties -Most of the properties that control internal settings have reasonable default values. However, -there are at least five properties that you will commonly want to control: +Most of the properties that control internal settings have reasonable default values. Some +of the most common options to set are: + + + + + + + + + + @@ -69,10 +105,12 @@ there are at least five properties that you will commonly want to control: @@ -81,7 +119,8 @@ there are at least five properties that you will commonly want to control: @@ -94,138 +133,151 @@ there are at least five properties that you will commonly want to control: comma-separated list of multiple directories on different disks. NOTE: In Spark 1.0 and later this will be overriden by SPARK_LOCAL_DIRS (Standalone, Mesos) or - LOCAL_DIRS (YARN) envrionment variables set by the cluster manager. + LOCAL_DIRS (YARN) environment variables set by the cluster manager. - - + +
      Property NameDefaultMeaning
      spark.app.name(none) + The name of your application. This will appear in the UI and in log data. +
      spark.master(none) + The cluster manager to connect to. See the list of + allowed master URL's. +
      spark.executor.memory 512m - Amount of memory to use per executor process, in the same format as JVM memory strings (e.g. - 512m, 2g). + Amount of memory to use per executor process, in the same format as JVM memory strings + (e.g. 512m, 2g).
      org.apache.spark.serializer.
      JavaSerializer
      Class to use for serializing objects that will be sent over the network or need to be cached - in serialized form. The default of Java serialization works with any Serializable Java object but is - quite slow, so we recommend using org.apache.spark.serializer.KryoSerializer - and configuring Kryo serialization when speed is necessary. Can be any subclass of - org.apache.spark.Serializer. + in serialized form. The default of Java serialization works with any Serializable Java object + but is quite slow, so we recommend using + org.apache.spark.serializer.KryoSerializer and configuring Kryo serialization + when speed is necessary. Can be any subclass of + + org.apache.spark.Serializer.
      If you use Kryo serialization, set this class to register your custom classes with Kryo. It should be set to a class that extends - KryoRegistrator. + + KryoRegistrator. See the tuning guide for more details.
      spark.cores.max(not set)spark.logConffalse - When running on a standalone deploy cluster or a - Mesos cluster in "coarse-grained" - sharing mode, the maximum amount of CPU cores to request for the application from - across the cluster (not from each machine). If not set, the default will be - spark.deploy.defaultCores on Spark's standalone cluster manager, or - infinite (all available cores) on Mesos. + Logs the effective SparkConf as INFO when a SparkContext is started.
      - Apart from these, the following properties are also available, and may be useful in some situations: +#### Runtime Environment - + + + + + + - - + + - - + + - - + + +
      Property NameDefaultMeaning
      spark.default.parallelismspark.executor.memory512m -
        -
      • Local mode: number of cores on the local machine
      • -
      • Mesos fine grained mode: 8
      • -
      • Others: total number of cores on all executor nodes or 2, whichever is larger
      • -
      + Amount of memory to use per executor process, in the same format as JVM memory strings + (e.g. 512m, 2g).
      spark.executor.extraJavaOptions(none) - Default number of tasks to use across the cluster for distributed shuffle operations - (groupByKey, reduceByKey, etc) when not set by user. + A string of extra JVM options to pass to executors. For instance, GC settings or other + logging. Note that it is illegal to set Spark properties or heap size settings with this + option. Spark properties should be set using a SparkConf object or the + spark-defaults.conf file used with the spark-submit script. Heap size settings can be set + with spark.executor.memory.
      spark.storage.memoryFraction0.6spark.executor.extraClassPath(none) - Fraction of Java heap to use for Spark's memory cache. This should not be larger than the "old" - generation of objects in the JVM, which by default is given 0.6 of the heap, but you can increase - it if you configure your own old generation size. + Extra classpath entries to append to the classpath of executors. This exists primarily + for backwards-compatibility with older versions of Spark. Users typically should not need + to set this option.
      spark.shuffle.memoryFraction0.3spark.executor.extraLibraryPath(none) - Fraction of Java heap to use for aggregation and cogroups during shuffles, if - spark.shuffle.spill is true. At any given time, the collective size of - all in-memory maps used for shuffles is bounded by this limit, beyond which the contents will - begin to spill to disk. If spills are often, consider increasing this value at the expense of - spark.storage.memoryFraction. + Set a special library path to use when launching executor JVM's.
      spark.storage.memoryMapThreshold8192spark.files.userClassPathFirstfalse - Size of a block, in bytes, above which Spark memory maps when reading a block from disk. - This prevents Spark from memory mapping very small blocks. In general, memory - mapping has high overhead for blocks close to or below the page size of the operating system. + (Experimental) Whether to give user-added jars precedence over Spark's own jars when + loading classes in Executors. This feature can be used to mitigate conflicts between + Spark's dependencies and user dependencies. It is currently an experimental feature.
      + +#### Shuffle Behavior + + - - + + - - + + - - + + - - + + - - + + - - + + - - + + + +
      Property NameDefaultMeaning
      spark.tachyonStore.baseDirSystem.getProperty("java.io.tmpdir")spark.shuffle.consolidateFilesfalse - Directories of the Tachyon File System that store RDDs. The Tachyon file system's URL is set by - spark.tachyonStore.url. It can also be a comma-separated list of multiple - directories on Tachyon file system. + If set to "true", consolidates intermediate files created during a shuffle. Creating fewer + files can improve filesystem performance for shuffles with large numbers of reduce tasks. It + is recommended to set this to "true" when using ext4 or xfs filesystems. On ext3, this option + might degrade performance on machines with many (>8) cores due to filesystem limitations.
      spark.tachyonStore.urltachyon://localhost:19998spark.shuffle.spilltrue - The URL of the underlying Tachyon file system in the TachyonStore. + If set to "true", limits the amount of memory used during reduces by spilling data out to disk. + This spilling threshold is specified by spark.shuffle.memoryFraction.
      spark.mesos.coarsefalsespark.shuffle.spill.compresstrue - If set to "true", runs over Mesos clusters in "coarse-grained" sharing mode, where Spark - acquires one long-lived Mesos task on each machine instead of one Mesos task per Spark task. - This gives lower-latency scheduling for short queries, but leaves resources in use for the whole - duration of the Spark job. + Whether to compress data spilled during shuffles. Compression will use + spark.io.compression.codec.
      spark.ui.port4040spark.shuffle.memoryFraction0.3 - Port for your application's dashboard, which shows memory and workload data + Fraction of Java heap to use for aggregation and cogroups during shuffles, if + spark.shuffle.spill is true. At any given time, the collective size of + all in-memory maps used for shuffles is bounded by this limit, beyond which the contents will + begin to spill to disk. If spills are often, consider increasing this value at the expense of + spark.storage.memoryFraction.
      spark.ui.retainedStages1000spark.shuffle.compresstrue - How many stages the Spark UI remembers before garbage collecting. + Whether to compress map output files. Generally a good idea. Compression will use + spark.io.compression.codec.
      spark.ui.filtersNonespark.shuffle.file.buffer.kb100 - Comma separated list of filter class names to apply to the Spark web ui. The filter should be a - standard javax servlet Filter. Parameters to each filter can also be specified by setting a - java system property of spark.<class name of filter>.params='param1=value1,param2=value2' - (e.g. -Dspark.ui.filters=com.test.filter1 -Dspark.com.test.filter1.params='param1=foo,param2=testing') + Size of the in-memory buffer for each shuffle file output stream, in kilobytes. These buffers + reduce the number of disk seeks and system calls made in creating intermediate shuffle files.
      spark.ui.acls.enablefalsespark.reducer.maxMbInFlight48 - Whether spark web ui acls should are enabled. If enabled, this checks to see if the user has - access permissions to view the web ui. See spark.ui.view.acls for more details. - Also note this requires the user to be known, if the user comes across as null no checks - are done. Filters can be used to authenticate and set the user. + Maximum size (in megabytes) of map outputs to fetch simultaneously from each reduce task. Since + each output requires us to create a buffer to receive it, this represents a fixed memory + overhead per reduce task, so keep it small unless you have a large amount of memory. +
      + +#### Spark UI + + + + + + - - + + @@ -236,19 +288,35 @@ Apart from these, the following properties are also available, and may be useful - - + + - - + + + + + + + +
      Property NameDefaultMeaning
      spark.ui.port4040 + Port for your application's dashboard, which shows memory and workload data
      spark.ui.view.aclsEmptyspark.ui.retainedStages1000 - Comma separated list of users that have view access to the spark web ui. By default only the - user that started the Spark job has view access. + How many stages the Spark UI remembers before garbage collecting.
      spark.shuffle.compresstruespark.eventLog.enabledfalse - Whether to compress map output files. Generally a good idea. + Whether to log Spark events, useful for reconstructing the Web UI after the application has + finished.
      spark.shuffle.spill.compresstruespark.eventLog.compressfalse - Whether to compress data spilled during shuffles. + Whether to compress logged events, if spark.eventLog.enabled is true.
      spark.eventLog.dirfile:///tmp/spark-events + Base directory in which Spark events are logged, if spark.eventLog.enabled is true. + Within this base directory, Spark creates a sub-directory for each application, and logs the + events specific to the application in this directory. Users may want to set this to + and HDFS directory so that history files can be read by the history server. +
      + +#### Compression and Serialization + + @@ -260,59 +328,46 @@ Apart from these, the following properties are also available, and may be useful - - - - - - - - - - - - + + - - + + @@ -329,21 +384,29 @@ Apart from these, the following properties are also available, and may be useful +
      Property NameDefaultMeaning
      spark.broadcast.compress truespark.rdd.compress false - Whether to compress serialized RDD partitions (e.g. for StorageLevel.MEMORY_ONLY_SER). - Can save substantial space at the cost of some extra CPU time. + Whether to compress serialized RDD partitions (e.g. for + StorageLevel.MEMORY_ONLY_SER). Can save substantial space at the cost of some + extra CPU time.
      spark.io.compression.codec org.apache.spark.io.
      LZFCompressionCodec
      - The codec used to compress internal data such as RDD partitions and shuffle outputs. By default, - Spark provides two codecs: org.apache.spark.io.LZFCompressionCodec and - org.apache.spark.io.SnappyCompressionCodec. + The codec used to compress internal data such as RDD partitions and shuffle outputs. + By default, Spark provides two codecs: org.apache.spark.io.LZFCompressionCodec + and org.apache.spark.io.SnappyCompressionCodec. Of these two choices, + Snappy offers faster compression and decompression, while LZF offers a better compression + ratio.
      spark.io.compression.snappy.block.size 32768 - Block size (in bytes) used in Snappy compression, in the case when Snappy compression codec is - used. -
      spark.scheduler.modeFIFO - The scheduling mode between - jobs submitted to the same SparkContext. Can be set to FAIR - to use fair sharing instead of queueing jobs one after another. Useful for - multi-user services. -
      spark.scheduler.revive.interval1000 - The interval length for the scheduler to revive the worker resource offers to run tasks. (in - milliseconds) + Block size (in bytes) used in Snappy compression, in the case when Snappy compression codec + is used.
      spark.reducer.maxMbInFlight48spark.closure.serializerorg.apache.spark.serializer.
      JavaSerializer
      - Maximum size (in megabytes) of map outputs to fetch simultaneously from each reduce task. Since - each output requires us to create a buffer to receive it, this represents a fixed memory - overhead per reduce task, so keep it small unless you have a large amount of memory. + Serializer class to use for closures. Currently only the Java serializer is supported.
      spark.closure.serializerorg.apache.spark.serializer.
      JavaSerializer
      spark.serializer.objectStreamReset10000 - Serializer class to use for closures. Currently only the Java serializer is supported. + When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches + objects to prevent writing redundant data, however that stops garbage collection of those + objects. By calling 'reset' you flush that info from the serializer, and allow old + objects to be collected. To turn off this periodic reset set it to a value <= 0. + By default it will reset the serializer every 10,000 objects.
      spark.kryoserializer.buffer.mb 2 - Maximum object size to allow within Kryo (the library needs to create a buffer at least as large - as the largest single object you'll serialize). Increase this if you get a "buffer limit + Maximum object size to allow within Kryo (the library needs to create a buffer at least as + large as the largest single object you'll serialize). Increase this if you get a "buffer limit exceeded" exception inside Kryo. Note that there will be one buffer per core on each worker.
      + +#### Execution Behavior + + - - + + @@ -354,73 +417,92 @@ Apart from these, the following properties are also available, and may be useful - - + + - - + + - - + + + + + + + + + + + + - - + + - - + + - - + + +
      Property NameDefaultMeaning
      spark.serializer.objectStreamReset10000spark.default.parallelism - When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches - objects to prevent writing redundant data, however that stops garbage collection of those - objects. By calling 'reset' you flush that info from the serializer, and allow old - objects to be collected. To turn off this periodic reset set it to a value <= 0. - By default it will reset the serializer every 10,000 objects. +
        +
      • Local mode: number of cores on the local machine
      • +
      • Mesos fine grained mode: 8
      • +
      • Others: total number of cores on all executor nodes or 2, whichever is larger
      • +
      +
      + Default number of tasks to use across the cluster for distributed shuffle operations + (groupByKey, reduceByKey, etc) when not set by user.
      spark.locality.wait3000spark.broadcast.blockSize4096 - Number of milliseconds to wait to launch a data-local task before giving up and launching it - on a less-local node. The same wait will be used to step through multiple locality levels - (process-local, node-local, rack-local and then any). It is also possible to customize the - waiting time for each level by setting spark.locality.wait.node, etc. - You should increase this setting if your tasks are long and see poor locality, but the - default usually works well. + Size of each piece of a block in kilobytes for TorrentBroadcastFactory. + Too large a value decreases parallelism during broadcast (makes it slower); however, if it is + too small, BlockManager might take a performance hit.
      spark.locality.wait.processspark.locality.waitspark.files.overwritefalse - Customize the locality wait for process locality. This affects tasks that attempt to access - cached data in a particular executor process. + Whether to overwrite files added through SparkContext.addFile() when the target file exists and + its contents do not match those of the source.
      spark.locality.wait.nodespark.locality.waitspark.files.fetchTimeoutfalse + Communication timeout to use when fetching files added through SparkContext.addFile() from + the driver. +
      spark.storage.memoryFraction0.6 + Fraction of Java heap to use for Spark's memory cache. This should not be larger than the "old" + generation of objects in the JVM, which by default is given 0.6 of the heap, but you can + increase it if you configure your own old generation size. +
      spark.tachyonStore.baseDirSystem.getProperty("java.io.tmpdir") - Customize the locality wait for node locality. For example, you can set this to 0 to skip - node locality and search immediately for rack locality (if your cluster has rack information). + Directories of the Tachyon File System that store RDDs. The Tachyon file system's URL is set by + spark.tachyonStore.url. It can also be a comma-separated list of multiple + directories on Tachyon file system.
      spark.locality.wait.rackspark.locality.waitspark.storage.memoryMapThreshold8192 - Customize the locality wait for rack locality. + Size of a block, in bytes, above which Spark memory maps when reading a block from disk. + This prevents Spark from memory mapping very small blocks. In general, memory + mapping has high overhead for blocks close to or below the page size of the operating system.
      spark.worker.timeout60spark.tachyonStore.urltachyon://localhost:19998 - Number of seconds after which the standalone deploy master considers a worker lost if it - receives no heartbeats. + The URL of the underlying Tachyon file system in the TachyonStore.
      spark.worker.cleanup.enabledfalsespark.cleaner.ttl(infinite) - Enable periodic cleanup of worker / application directories. Note that this only affects - standalone mode, as YARN works differently. Applications directories are cleaned up regardless - of whether the application is still running. + Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks + generated, etc.). Periodic cleanups will ensure that metadata older than this duration will be + forgotten. This is useful for running Spark for many hours / days (for example, running 24/7 in + case of Spark Streaming applications). Note that any RDD that persists in memory for more than + this duration will be cleared as well.
      + +#### Networking + + - - + + - - + + @@ -454,8 +536,8 @@ Apart from these, the following properties are also available, and may be useful This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart beat pause in seconds for akka. This can be used to control sensitivity to gc pauses. Tune this in - combination of `spark.akka.heartbeat.interval` and `spark.akka.failure-detector.threshold` if - you need to. + combination of `spark.akka.heartbeat.interval` and `spark.akka.failure-detector.threshold` + if you need to. @@ -476,55 +558,23 @@ Apart from these, the following properties are also available, and may be useful enabled again, if you plan to use this feature (Not recommended). A larger interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and - `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using failure - detector can be, a sensistive failure detector can help evict rogue executors really quick. - However this is usually not the case as gc pauses and network lags are expected in a real spark - cluster. Apart from that enabling this leads to a lot of exchanges of heart beats between nodes - leading to flooding the network with those. - - - - - - - - - - - - - - - - - - - - - +
      Property NameDefaultMeaning
      spark.worker.cleanup.interval1800 (30 minutes)spark.driver.host(local hostname) - Controls the interval, in seconds, at which the worker cleans up old application work dirs - on the local machine. + Hostname or IP address for the driver to listen on.
      spark.worker.cleanup.appDataTtl7 * 24 * 3600 (7 days)spark.driver.port(random) - The number of seconds to retain application work directories on each worker. This is a Time To - Live and should depend on the amount of available disk space you have. Application logs and - jars are downloaded to each application work dir. Over time, the work dirs can quickly fill up - disk space, especially if you run jobs very frequently. + Port for the driver to listen on.
      spark.driver.host(local hostname) - Hostname or IP address for the driver to listen on. -
      spark.driver.port(random) - Port for the driver to listen on. -
      spark.cleaner.ttl(infinite) - Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks - generated, etc.). Periodic cleanups will ensure that metadata older than this duration will be - forgotten. This is useful for running Spark for many hours / days (for example, running 24/7 in - case of Spark Streaming applications). Note that any RDD that persists in memory for more than - this duration will be cleared as well. -
      spark.streaming.blockInterval200 - Interval (milliseconds) at which data received by Spark Streaming receivers is coalesced - into blocks of data before storing them in Spark. + `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using + failure detector can be, a sensistive failure detector can help evict rogue executors really + quick. However this is usually not the case as gc pauses and network lags are expected in a + real Spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats + between nodes leading to flooding the network with those.
      + +#### Scheduling + + - - + + @@ -536,39 +586,36 @@ Apart from these, the following properties are also available, and may be useful - - - - - - - - + + - - + + - - + + @@ -601,91 +648,59 @@ Apart from these, the following properties are also available, and may be useful - - - - - - - - - - - - - - - - - - - - - - + + - - + + - - + + - - + + - - + + +
      Property NameDefaultMeaning
      spark.streaming.unpersisttruespark.task.cpus1 - Force RDDs generated and persisted by Spark Streaming to be automatically unpersisted from - Spark's memory. The raw input data received by Spark Streaming is also automatically cleared. - Setting this to false will allow the raw data and persisted RDDs to be accessible outside the - streaming application as they will not be cleared automatically. But it comes at the cost of - higher memory usage in Spark. + Number of cores to allocate for each task.
      spark.broadcast.blockSize4096 - Size of each piece of a block in kilobytes for TorrentBroadcastFactory. - Too large a value decreases parallelism during broadcast (makes it slower); however, if it is - too small, BlockManager might take a performance hit. -
      spark.shuffle.consolidateFilesfalsespark.scheduler.modeFIFO - If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files - can improve filesystem performance for shuffles with large numbers of reduce tasks. It is - recommended to set this to "true" when using ext4 or xfs filesystems. On ext3, this option might - degrade performance on machines with many (>8) cores due to filesystem limitations. + The scheduling mode between + jobs submitted to the same SparkContext. Can be set to FAIR + to use fair sharing instead of queueing jobs one after another. Useful for + multi-user services.
      spark.shuffle.file.buffer.kb100spark.cores.max(not set) - Size of the in-memory buffer for each shuffle file output stream, in kilobytes. These buffers - reduce the number of disk seeks and system calls made in creating intermediate shuffle files. + When running on a standalone deploy cluster or a + Mesos cluster in "coarse-grained" + sharing mode, the maximum amount of CPU cores to request for the application from + across the cluster (not from each machine). If not set, the default will be + spark.deploy.defaultCores on Spark's standalone cluster manager, or + infinite (all available cores) on Mesos.
      spark.shuffle.spilltruespark.mesos.coarsefalse - If set to "true", limits the amount of memory used during reduces by spilling data out to disk. - This spilling threshold is specified by spark.shuffle.memoryFraction. + If set to "true", runs over Mesos clusters in + "coarse-grained" sharing mode, + where Spark acquires one long-lived Mesos task on each machine instead of one Mesos task per + Spark task. This gives lower-latency scheduling for short queries, but leaves resources in use + for the whole duration of the Spark job.
      spark.logConffalse - Whether to log the supplied SparkConf as INFO at start of spark context. -
      spark.eventLog.enabledfalse - Whether to log spark events, useful for reconstructing the Web UI after the application has - finished. -
      spark.eventLog.compressfalse - Whether to compress logged events, if spark.eventLog.enabled is true. -
      spark.eventLog.dirfile:///tmp/spark-events - Base directory in which spark events are logged, if spark.eventLog.enabled is true. - Within this base directory, Spark creates a sub-directory for each application, and logs the - events specific to the application in this directory. -
      spark.deploy.spreadOuttruespark.locality.wait3000 - Whether the standalone cluster manager should spread applications out across nodes or try to - consolidate them onto as few nodes as possible. Spreading out is usually better for data - locality in HDFS, but consolidating is more efficient for compute-intensive workloads.
      - Note: this setting needs to be configured in the standalone cluster master, not in - individual applications; you can set it through SPARK_MASTER_OPTS in - spark-env.sh. + Number of milliseconds to wait to launch a data-local task before giving up and launching it + on a less-local node. The same wait will be used to step through multiple locality levels + (process-local, node-local, rack-local and then any). It is also possible to customize the + waiting time for each level by setting spark.locality.wait.node, etc. + You should increase this setting if your tasks are long and see poor locality, but the + default usually works well.
      spark.deploy.defaultCores(infinite)spark.locality.wait.processspark.locality.wait - Default number of cores to give to applications in Spark's standalone mode if they don't set - spark.cores.max. If not set, applications always get all available cores unless - they configure spark.cores.max themselves. Set this lower on a shared cluster to - prevent users from grabbing the whole cluster by default.
      Note: this setting needs - to be configured in the standalone cluster master, not in individual applications; you can set - it through SPARK_MASTER_OPTS in spark-env.sh. + Customize the locality wait for process locality. This affects tasks that attempt to access + cached data in a particular executor process.
      spark.files.overwritefalsespark.locality.wait.nodespark.locality.wait - Whether to overwrite files added through SparkContext.addFile() when the target file exists and - its contents do not match those of the source. + Customize the locality wait for node locality. For example, you can set this to 0 to skip + node locality and search immediately for rack locality (if your cluster has rack information).
      spark.files.fetchTimeoutfalsespark.locality.wait.rackspark.locality.wait - Communication timeout to use when fetching files added through SparkContext.addFile() from - the driver. + Customize the locality wait for rack locality.
      spark.files.userClassPathFirstfalsespark.scheduler.revive.interval1000 - (Experimental) Whether to give user-added jars precedence over Spark's own jars when - loading classes in Executors. This feature can be used to mitigate conflicts between - Spark's dependencies and user dependencies. It is currently an experimental feature. + The interval length for the scheduler to revive the worker resource offers to run tasks. + (in milliseconds)
      + +#### Security + + @@ -693,7 +708,7 @@ Apart from these, the following properties are also available, and may be useful @@ -705,42 +720,71 @@ Apart from these, the following properties are also available, and may be useful - - + + - - + + - - + + +
      Property NameDefaultMeaning
      spark.authenticate false - Whether spark authenticates its internal connections. See spark.authenticate.secret - if not running on Yarn. + Whether Spark authenticates its internal connections. See + spark.authenticate.secret if not running on YARN.
      None Set the secret key used for Spark to authenticate between components. This needs to be set if - not running on Yarn and authentication is enabled. + not running on YARN and authentication is enabled.
      spark.task.cpus1spark.ui.filtersNone - Number of cores to allocate for each task. + Comma separated list of filter class names to apply to the Spark web UI. The filter should be a + standard + javax servlet Filter. Parameters to each filter can also be specified by setting a + java system property of:
      + spark.<class name of filter>.params='param1=value1,param2=value2'
      + For example:
      + -Dspark.ui.filters=com.test.filter1
      + -Dspark.com.test.filter1.params='param1=foo,param2=testing'
      spark.executor.extraJavaOptions(none)spark.ui.acls.enablefalse - A string of extra JVM options to pass to executors. For instance, GC settings or other - logging. Note that it is illegal to set Spark properties or heap size settings with this - option. Spark properties should be set using a SparkConf object or the - spark-defaults.conf file used with the spark-submit script. Heap size settings can be set - with spark.executor.memory. + Whether Spark web ui acls should are enabled. If enabled, this checks to see if the user has + access permissions to view the web ui. See spark.ui.view.acls for more details. + Also note this requires the user to be known, if the user comes across as null no checks + are done. Filters can be used to authenticate and set the user.
      spark.executor.extraClassPath(none)spark.ui.view.aclsEmpty - Extra classpath entries to append to the classpath of executors. This exists primarily - for backwards-compatibility with older versions of Spark. Users typically should not need - to set this option. + Comma separated list of users that have view access to the Spark web ui. By default only the + user that started the Spark job has view access.
      + +#### Spark Streaming + + - - + + + + + + + -
      Property NameDefaultMeaning
      spark.executor.extraLibraryPath(none)spark.streaming.blockInterval200 - Set a special library path to use when launching executor JVM's. + Interval (milliseconds) at which data received by Spark Streaming receivers is coalesced + into blocks of data before storing them in Spark. +
      spark.streaming.unpersisttrue + Force RDDs generated and persisted by Spark Streaming to be automatically unpersisted from + Spark's memory. The raw input data received by Spark Streaming is also automatically cleared. + Setting this to false will allow the raw data and persisted RDDs to be accessible outside the + streaming application as they will not be cleared automatically. But it comes at the cost of + higher memory usage in Spark.
      +#### Cluster Managers (YARN, Mesos, Standalone) +Each cluster manager in Spark has additional configuration options. Configurations +can be found on the pages for each mode: + + * [YARN](running-on-yarn.html#configuration) + * [Mesos](running-on-mesos.html) + * [Standalone Mode](spark-standalone.html#cluster-launch-scripts) + # Environment Variables Certain Spark settings can be configured through environment variables, which are read from the @@ -774,104 +818,15 @@ The following variables can be set in `spark-env.sh`: -In addition to the above, there are also options for setting up the Spark [standalone cluster -scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores to use on each -machine and maximum memory. +In addition to the above, there are also options for setting up the Spark +[standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores +to use on each machine and maximum memory. -Since `spark-env.sh` is a shell script, some of these can be set programmatically -- for example, -you might compute `SPARK_LOCAL_IP` by looking up the IP of a specific network interface. +Since `spark-env.sh` is a shell script, some of these can be set programmatically -- for example, you might +compute `SPARK_LOCAL_IP` by looking up the IP of a specific network interface. # Configuring Logging Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can configure it by adding a `log4j.properties` file in the `conf` directory. One way to start is to copy the existing `log4j.properties.template` located there. - -# Configuring Ports for Network Security - -Spark makes heavy use of the network, and some environments have strict requirements for using tight -firewall settings. Below are the primary ports that Spark uses for its communication and how to -configure those ports. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
      FromToDefault PortPurposeConfiguration - SettingNotes
      BrowserStandalone Cluster Master8080Web UImaster.ui.portJetty-based
      BrowserWorker8081Web UIworker.ui.portJetty-based
      BrowserDriver4040Web UIspark.ui.portJetty-based
      BrowserHistory Server18080Web UIspark.history.ui.portJetty-based
      ApplicationStandalone Cluster Master7077Submit job to clusterspark.driver.portAkka-based. Set to "0" to choose a port randomly
      WorkerStandalone Cluster Master7077Join clusterspark.driver.portAkka-based. Set to "0" to choose a port randomly
      ApplicationWorker(random)Join clusterSPARK_WORKER_PORT (standalone cluster)Akka-based
      Driver and other WorkersWorker(random) -
        -
      • File server for file and jars
      • -
      • Http Broadcast
      • -
      • Class file server (Spark Shell only)
      • -
      -
      NoneJetty-based. Each of these services starts on a random port that cannot be configured
      diff --git a/docs/quick-start.md b/docs/quick-start.md index 33a0df1036424..20e17ebf703fc 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -252,11 +252,11 @@ we initialize a SparkContext as part of the program. We pass the SparkContext constructor a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object which contains information about our -application. We also call sc.addJar to make sure that when our application is launched in cluster -mode, the jar file containing it will be shipped automatically to worker nodes. +application. -This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` -which explains that Spark is a dependency. This file also adds a repository that Spark depends on: +Our application depends on the Spark API, so we'll also include an sbt configuration file, +`simple.sbt` which explains that Spark is a dependency. This file also adds a repository that +Spark depends on: {% highlight scala %} name := "Simple Project" diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index eb3211b6b0e4e..dca80a9a69614 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -93,7 +93,15 @@ You can optionally configure the cluster further by setting environment variable SPARK_MASTER_OPTS - Configuration properties that apply only to the master in the form "-Dx=y" (default: none). + Configuration properties that apply only to the master in the form "-Dx=y" (default: none). See below for a list of possible options. + + + SPARK_LOCAL_DIRS + + Directory to use for "scratch" space in Spark, including map output files and RDDs that get + stored on disk. This should be on a fast, local disk in your system. It can also be a + comma-separated list of multiple directories on different disks. + SPARK_WORKER_CORES @@ -126,7 +134,7 @@ You can optionally configure the cluster further by setting environment variable SPARK_WORKER_OPTS - Configuration properties that apply only to the worker in the form "-Dx=y" (default: none). + Configuration properties that apply only to the worker in the form "-Dx=y" (default: none). See below for a list of possible options. SPARK_DAEMON_MEMORY @@ -144,6 +152,73 @@ You can optionally configure the cluster further by setting environment variable **Note:** The launch scripts do not currently support Windows. To run a Spark cluster on Windows, start the master and workers by hand. +SPARK_MASTER_OPTS supports the following system properties: + + + + + + + + + + + + + + + + + + +
      Property NameDefaultMeaning
      spark.deploy.spreadOuttrue + Whether the standalone cluster manager should spread applications out across nodes or try + to consolidate them onto as few nodes as possible. Spreading out is usually better for + data locality in HDFS, but consolidating is more efficient for compute-intensive workloads.
      +
      spark.deploy.defaultCores(infinite) + Default number of cores to give to applications in Spark's standalone mode if they don't + set spark.cores.max. If not set, applications always get all available + cores unless they configure spark.cores.max themselves. + Set this lower on a shared cluster to prevent users from grabbing + the whole cluster by default.
      +
      spark.worker.timeout60 + Number of seconds after which the standalone deploy master considers a worker lost if it + receives no heartbeats. +
      + +SPARK_WORKER_OPTS supports the following system properties: + + + + + + + + + + + + + + + + + + +
      Property NameDefaultMeaning
      spark.worker.cleanup.enabledfalse + Enable periodic cleanup of worker / application directories. Note that this only affects standalone + mode, as YARN works differently. Applications directories are cleaned up regardless of whether + the application is still running. +
      spark.worker.cleanup.interval1800 (30 minutes) + Controls the interval, in seconds, at which the worker cleans up old application work dirs + on the local machine. +
      spark.worker.cleanup.appDataTtl7 * 24 * 3600 (7 days) + The number of seconds to retain application work directories on each worker. This is a Time To Live + and should depend on the amount of available disk space you have. Application logs and jars are + downloaded to each application work dir. Over time, the work dirs can quickly fill up disk space, + especially if you run jobs very frequently. +
      + # Connecting an Application to the Cluster To run an application on the Spark cluster, simply pass the `spark://IP:PORT` URL of the master as to the [`SparkContext` @@ -212,6 +287,94 @@ In addition, detailed log output for each job is also written to the work direct You can run Spark alongside your existing Hadoop cluster by just launching it as a separate service on the same machines. To access Hadoop data from Spark, just use a hdfs:// URL (typically `hdfs://:9000/path`, but you can find the right URL on your Hadoop Namenode's web UI). Alternatively, you can set up a separate cluster for Spark, and still have it access HDFS over the network; this will be slower than disk-local access, but may not be a concern if you are still running in the same local area network (e.g. you place a few Spark machines on each rack that you have Hadoop on). +# Configuring Ports for Network Security + +Spark makes heavy use of the network, and some environments have strict requirements for using tight +firewall settings. Below are the primary ports that Spark uses for its communication and how to +configure those ports. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      FromToDefault PortPurposeConfiguration + SettingNotes
      BrowserStandalone Cluster Master8080Web UImaster.ui.portJetty-based
      BrowserDriver4040Web UIspark.ui.portJetty-based
      BrowserHistory Server18080Web UIspark.history.ui.portJetty-based
      BrowserWorker8081Web UIworker.ui.portJetty-based
      ApplicationStandalone Cluster Master7077Submit job to clusterspark.driver.portAkka-based. Set to "0" to choose a port randomly
      WorkerStandalone Cluster Master7077Join clusterspark.driver.portAkka-based. Set to "0" to choose a port randomly
      ApplicationWorker(random)Join clusterSPARK_WORKER_PORT (standalone cluster)Akka-based
      Driver and other WorkersWorker(random) +
        +
      • File server for file and jars
      • +
      • Http Broadcast
      • +
      • Class file server (Spark Shell only)
      • +
      +
      NoneJetty-based. Each of these services starts on a random port that cannot be configured
      + # High Availability By default, standalone scheduling clusters are resilient to Worker failures (insofar as Spark itself is resilient to losing work by moving it to other workers). However, the scheduler uses a Master to make scheduling decisions, and this (by default) creates a single point of failure: if the Master crashes, no new applications can be created. In order to circumvent this, we have two high availability schemes, detailed below. From 4312cf0bade82d4b54eef80e637953e1633edc3e Mon Sep 17 00:00:00 2001 From: David Lemieux Date: Wed, 28 May 2014 15:50:35 -0700 Subject: [PATCH 444/641] Spark 1916 The changes could be ported back to 0.9 as well. Changing in.read to in.readFully to read the whole input stream rather than the first 1020 bytes. This should ok considering that Flume caps the body size to 32K by default. Author: David Lemieux Closes #865 from lemieud/SPARK-1916 and squashes the following commits: a265673 [David Lemieux] Updated SparkFlumeEvent to read the whole stream rather than the first X bytes. (cherry picked from commit 0b769b73fb7ae314325857138a2d3138ed157908) Signed-off-by: Patrick Wendell --- .../org/apache/spark/streaming/flume/FlumeInputDStream.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index df7605fe579f8..5be33f1d5c428 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -63,7 +63,7 @@ class SparkFlumeEvent() extends Externalizable { def readExternal(in: ObjectInput) { val bodyLength = in.readInt() val bodyBuff = new Array[Byte](bodyLength) - in.read(bodyBuff) + in.readFully(bodyBuff) val numHeaders = in.readInt() val headers = new java.util.HashMap[CharSequence, CharSequence] From 4dbb27b0cf4eb67c92aad2c1158616312f5a54e6 Mon Sep 17 00:00:00 2001 From: witgo Date: Wed, 28 May 2014 15:57:05 -0700 Subject: [PATCH 445/641] [SPARK-1712]: TaskDescription instance is too big causes Spark to hang Author: witgo Closes #694 from witgo/SPARK-1712_new and squashes the following commits: 0f52483 [witgo] review commit 83ce29b [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new 52e6752 [witgo] reset test SparkContext 63636b6 [witgo] review commit 44a59ee [witgo] review commit 3b6d48c [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new 926bd6a [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new 9a5cfad [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new 03cc562 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new b0930b0 [witgo] review commit b1174bd [witgo] merge master f76679b [witgo] merge master 689495d [witgo] fix scala style bug 1d35c3c [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new 062c182 [witgo] fix small bug for code style 0a428cf [witgo] add unit tests 158b2dc [witgo] review commit 4afe71d [witgo] review commit 9e4ffa7 [witgo] review commit 1d35c7d [witgo] fix hang 7965580 [witgo] fix Statement order 0e29eac [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new 3ea1ca1 [witgo] remove duplicate serialize 743a7ad [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new 86e2048 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new 2a89adc [witgo] SPARK-1712: TaskDescription instance is too big causes Spark to hang --- .../CoarseGrainedExecutorBackend.scala | 9 ++-- .../cluster/CoarseGrainedClusterMessage.scala | 2 +- .../CoarseGrainedSchedulerBackend.scala | 27 ++++++++++-- .../CoarseGrainedSchedulerBackendSuite.scala | 43 +++++++++++++++++++ 4 files changed, 73 insertions(+), 8 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 84aec65b7765d..2279d77c91c89 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -22,11 +22,12 @@ import java.nio.ByteBuffer import akka.actor._ import akka.remote._ -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SparkEnv, Logging, SecurityManager, SparkConf} import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ +import org.apache.spark.scheduler.TaskDescription import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class CoarseGrainedExecutorBackend( @@ -61,12 +62,14 @@ private[spark] class CoarseGrainedExecutorBackend( logError("Slave registration failed: " + message) System.exit(1) - case LaunchTask(taskDesc) => - logInfo("Got assigned task " + taskDesc.taskId) + case LaunchTask(data) => if (executor == null) { logError("Received LaunchTask command but executor was null") System.exit(1) } else { + val ser = SparkEnv.get.closureSerializer.newInstance() + val taskDesc = ser.deserialize[TaskDescription](data.value) + logInfo("Got assigned task " + taskDesc.taskId) executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index ddbc74e82ac49..ca74069ef885c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -28,7 +28,7 @@ private[spark] sealed trait CoarseGrainedClusterMessage extends Serializable private[spark] object CoarseGrainedClusterMessages { // Driver to executors - case class LaunchTask(task: TaskDescription) extends CoarseGrainedClusterMessage + case class LaunchTask(data: SerializableBuffer) extends CoarseGrainedClusterMessage case class KillTask(taskId: Long, executor: String, interruptThread: Boolean) extends CoarseGrainedClusterMessage diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index a6d6b3d26a3c6..e47a060683a2d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -27,10 +27,10 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.{Logging, SparkException, TaskState} +import org.apache.spark.{SparkEnv, Logging, SparkException, TaskState} import org.apache.spark.scheduler.{SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{SerializableBuffer, AkkaUtils, Utils} /** * A scheduler backend that waits for coarse grained executors to connect to it through Akka. @@ -48,6 +48,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A var totalCoreCount = new AtomicInteger(0) val conf = scheduler.sc.conf private val timeout = AkkaUtils.askTimeout(conf) + private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor { private val executorActor = new HashMap[String, ActorRef] @@ -140,8 +141,26 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A // Launch tasks returned by a set of resource offers def launchTasks(tasks: Seq[Seq[TaskDescription]]) { for (task <- tasks.flatten) { - freeCores(task.executorId) -= scheduler.CPUS_PER_TASK - executorActor(task.executorId) ! LaunchTask(task) + val ser = SparkEnv.get.closureSerializer.newInstance() + val serializedTask = ser.serialize(task) + if (serializedTask.limit >= akkaFrameSize - 1024) { + val taskSetId = scheduler.taskIdToTaskSetId(task.taskId) + scheduler.activeTaskSets.get(taskSetId).foreach { taskSet => + try { + var msg = "Serialized task %s:%d was %d bytes which " + + "exceeds spark.akka.frameSize (%d bytes). " + + "Consider using broadcast variables for large values." + msg = msg.format(task.taskId, task.index, serializedTask.limit, akkaFrameSize) + taskSet.abort(msg) + } catch { + case e: Exception => logError("Exception in error callback", e) + } + } + } + else { + freeCores(task.executorId) -= scheduler.CPUS_PER_TASK + executorActor(task.executorId) ! LaunchTask(new SerializableBuffer(serializedTask)) + } } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala new file mode 100644 index 0000000000000..efef9d26dadca --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -0,0 +1,43 @@ +/* + * 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.scheduler + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkException, SparkContext} +import org.apache.spark.util.{SerializableBuffer, AkkaUtils} + +import org.scalatest.FunSuite + +class CoarseGrainedSchedulerBackendSuite extends FunSuite with LocalSparkContext { + + test("serialized task larger than akka frame size") { + val conf = new SparkConf + conf.set("spark.akka.frameSize","1") + conf.set("spark.default.parallelism","1") + sc = new SparkContext("local-cluster[2 , 1 , 512]", "test", conf) + val frameSize = AkkaUtils.maxFrameSizeBytes(sc.conf) + val buffer = new SerializableBuffer(java.nio.ByteBuffer.allocate(2 * frameSize)) + val larger = sc.parallelize(Seq(buffer)) + val thrown = intercept[SparkException] { + larger.collect() + } + assert(thrown.getMessage.contains("Consider using broadcast variables for large values")) + val smaller = sc.parallelize(1 to 4).collect() + assert(smaller.size === 4) + } + +} From 9cff1dd25abc5e848720d853172ed42e35376fd0 Mon Sep 17 00:00:00 2001 From: Jyotiska NK Date: Wed, 28 May 2014 23:08:39 -0700 Subject: [PATCH 446/641] Added doctest and method description in context.py Added doctest for method textFile and description for methods _initialize_context and _ensure_initialized in context.py Author: Jyotiska NK Closes #187 from jyotiska/pyspark_context and squashes the following commits: 356f945 [Jyotiska NK] Added doctest for textFile method in context.py 5b23686 [Jyotiska NK] Updated context.py with method descriptions --- python/pyspark/context.py | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 27b440d73bdc3..56746cb7aab3d 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -173,12 +173,18 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, self._temp_dir = \ self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir).getAbsolutePath() - # Initialize SparkContext in function to allow subclass specific initialization def _initialize_context(self, jconf): + """ + Initialize SparkContext in function to allow subclass specific initialization + """ return self._jvm.JavaSparkContext(jconf) @classmethod def _ensure_initialized(cls, instance=None, gateway=None): + """ + Checks whether a SparkContext is initialized or not. + Throws error if a SparkContext is already running. + """ with SparkContext._lock: if not SparkContext._gateway: SparkContext._gateway = gateway or launch_gateway() @@ -270,6 +276,13 @@ def textFile(self, name, minPartitions=None): Read a text file from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI, and return it as an RDD of Strings. + + >>> path = os.path.join(tempdir, "sample-text.txt") + >>> with open(path, "w") as testFile: + ... testFile.write("Hello world!") + >>> textFile = sc.textFile(path) + >>> textFile.collect() + [u'Hello world!'] """ minPartitions = minPartitions or min(self.defaultParallelism, 2) return RDD(self._jsc.textFile(name, minPartitions), self, From 60b89fe6b09ff896a30d74204876da883e307de7 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 29 May 2014 09:07:39 -0700 Subject: [PATCH 447/641] SPARK-1935: Explicitly add commons-codec 1.5 as a dependency. Author: Yin Huai Closes #889 from yhuai/SPARK-1935 and squashes the following commits: 7d50ef1 [Yin Huai] Explicitly add commons-codec 1.5 as a dependency. --- pom.xml | 5 +++++ project/SparkBuild.scala | 1 + 2 files changed, 6 insertions(+) diff --git a/pom.xml b/pom.xml index 86264d1132ec4..7bf9f135fd340 100644 --- a/pom.xml +++ b/pom.xml @@ -240,6 +240,11 @@ commons-lang3 3.3.2 + + commons-codec + commons-codec + 1.5 + com.google.code.findbugs jsr305 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b9d92340ff75b..8ef1e91f609fb 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -348,6 +348,7 @@ object SparkBuild extends Build { "org.apache.mesos" % "mesos" % "0.18.1" classifier("shaded-protobuf") exclude("com.google.protobuf", "protobuf-java"), "commons-net" % "commons-net" % "2.2", "net.java.dev.jets3t" % "jets3t" % jets3tVersion excludeAll(excludeCommonsLogging), + "commons-codec" % "commons-codec" % "1.5", // Prevent jets3t from including the older version of commons-codec "org.apache.derby" % "derby" % "10.4.2.0" % "test", "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm), "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeJBossNetty), From 8f7141fbc015addb314e1d5801085587b5cbb171 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 29 May 2014 15:24:03 -0700 Subject: [PATCH 448/641] [SPARK-1368][SQL] Optimized HiveTableScan JIRA issue: [SPARK-1368](https://issues.apache.org/jira/browse/SPARK-1368) This PR introduces two major updates: - Replaced FP style code with `while` loop and reusable `GenericMutableRow` object in critical path of `HiveTableScan`. - Using `ColumnProjectionUtils` to help optimizing RCFile and ORC column pruning. My quick micro benchmark suggests these two optimizations made the optimized version 2x and 2.5x faster when scanning CSV table and RCFile table respectively: ``` Original: [info] CSV: 27676 ms, RCFile: 26415 ms [info] CSV: 27703 ms, RCFile: 26029 ms [info] CSV: 27511 ms, RCFile: 25962 ms Optimized: [info] CSV: 13820 ms, RCFile: 10402 ms [info] CSV: 14158 ms, RCFile: 10691 ms [info] CSV: 13606 ms, RCFile: 10346 ms ``` The micro benchmark loads a 609MB CVS file (structurally similar to the `src` test table) into a normal Hive table with `LazySimpleSerDe` and a RCFile table, then scans these tables respectively. Preparation code: ```scala package org.apache.spark.examples.sql.hive import org.apache.spark.sql.hive.LocalHiveContext import org.apache.spark.{SparkConf, SparkContext} object HiveTableScanPrepare extends App { val sparkContext = new SparkContext( new SparkConf() .setMaster("local") .setAppName(getClass.getSimpleName.stripSuffix("$"))) val hiveContext = new LocalHiveContext(sparkContext) import hiveContext._ hql("drop table scan_csv") hql("drop table scan_rcfile") hql("""create table scan_csv (key int, value string) | row format serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' | with serdeproperties ('field.delim'=',') """.stripMargin) hql(s"""load data local inpath "${args(0)}" into table scan_csv""") hql("""create table scan_rcfile (key int, value string) | row format serde 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' |stored as | inputformat 'org.apache.hadoop.hive.ql.io.RCFileInputFormat' | outputformat 'org.apache.hadoop.hive.ql.io.RCFileOutputFormat' """.stripMargin) hql( """ |from scan_csv |insert overwrite table scan_rcfile |select scan_csv.key, scan_csv.value """.stripMargin) } ``` Benchmark code: ```scala package org.apache.spark.examples.sql.hive import org.apache.spark.sql.hive.LocalHiveContext import org.apache.spark.{SparkConf, SparkContext} object HiveTableScanBenchmark extends App { val sparkContext = new SparkContext( new SparkConf() .setMaster("local") .setAppName(getClass.getSimpleName.stripSuffix("$"))) val hiveContext = new LocalHiveContext(sparkContext) import hiveContext._ val scanCsv = hql("select key from scan_csv") val scanRcfile = hql("select key from scan_rcfile") val csvDuration = benchmark(scanCsv.count()) val rcfileDuration = benchmark(scanRcfile.count()) println(s"CSV: $csvDuration ms, RCFile: $rcfileDuration ms") def benchmark(f: => Unit) = { val begin = System.currentTimeMillis() f val end = System.currentTimeMillis() end - begin } } ``` @marmbrus Please help review, thanks! Author: Cheng Lian Closes #758 from liancheng/fastHiveTableScan and squashes the following commits: 4241a19 [Cheng Lian] Distinguishes sorted and possibly not sorted operations more accurately in HiveComparisonTest cf640d8 [Cheng Lian] More HiveTableScan optimisations: bf0e7dc [Cheng Lian] Added SortedOperation pattern to match *some* definitely sorted operations and avoid some sorting cost in HiveComparisonTest. 6d1c642 [Cheng Lian] Using ColumnProjectionUtils to optimise RCFile and ORC column pruning eb62fd3 [Cheng Lian] [SPARK-1368] Optimized HiveTableScan --- .../spark/sql/execution/Aggregate.scala | 2 +- .../apache/spark/sql/hive/hiveOperators.scala | 97 ++++++++++++++++--- .../hive/execution/HiveComparisonTest.scala | 25 ++--- 3 files changed, 96 insertions(+), 28 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala index 36b3b956da96c..604914e547790 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala @@ -116,7 +116,7 @@ case class Aggregate( */ @transient private[this] lazy val resultMap = - (computedAggregates.map { agg => agg.unbound -> agg.resultAttribute} ++ namedGroups).toMap + (computedAggregates.map { agg => agg.unbound -> agg.resultAttribute } ++ namedGroups).toMap /** * Substituted version of aggregateExpressions expressions which are used to compute final diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala index 96faebc5a8687..f141139ef46a8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala @@ -18,15 +18,18 @@ package org.apache.spark.sql.hive.execution import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} +import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.MetaStoreUtils import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Hive} import org.apache.hadoop.hive.ql.plan.{TableDesc, FileSinkDesc} -import org.apache.hadoop.hive.serde2.Serializer +import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveDecimalObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharObjectInspector +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils +import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Serializer} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred._ @@ -37,6 +40,7 @@ import org.apache.spark.sql.catalyst.types.{BooleanType, DataType} import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive._ import org.apache.spark.{TaskContext, SparkException} +import org.apache.spark.util.MutablePair /* Implicits */ import scala.collection.JavaConversions._ @@ -94,7 +98,7 @@ case class HiveTableScan( (_: Any, partitionKeys: Array[String]) => { val value = partitionKeys(ordinal) val dataType = relation.partitionKeys(ordinal).dataType - castFromString(value, dataType) + unwrapHiveData(castFromString(value, dataType)) } } else { val ref = objectInspector.getAllStructFieldRefs @@ -102,16 +106,55 @@ case class HiveTableScan( .getOrElse(sys.error(s"Can't find attribute $a")) (row: Any, _: Array[String]) => { val data = objectInspector.getStructFieldData(row, ref) - unwrapData(data, ref.getFieldObjectInspector) + unwrapHiveData(unwrapData(data, ref.getFieldObjectInspector)) } } } } + private def unwrapHiveData(value: Any) = value match { + case maybeNull: String if maybeNull.toLowerCase == "null" => null + case varchar: HiveVarchar => varchar.getValue + case decimal: HiveDecimal => BigDecimal(decimal.bigDecimalValue) + case other => other + } + private def castFromString(value: String, dataType: DataType) = { Cast(Literal(value), dataType).eval(null) } + private def addColumnMetadataToConf(hiveConf: HiveConf) { + // Specifies IDs and internal names of columns to be scanned. + val neededColumnIDs = attributes.map(a => relation.output.indexWhere(_.name == a.name): Integer) + val columnInternalNames = neededColumnIDs.map(HiveConf.getColumnInternalName(_)).mkString(",") + + if (attributes.size == relation.output.size) { + ColumnProjectionUtils.setFullyReadColumns(hiveConf) + } else { + ColumnProjectionUtils.appendReadColumnIDs(hiveConf, neededColumnIDs) + } + + ColumnProjectionUtils.appendReadColumnNames(hiveConf, attributes.map(_.name)) + + // Specifies types and object inspectors of columns to be scanned. + val structOI = ObjectInspectorUtils + .getStandardObjectInspector( + relation.tableDesc.getDeserializer.getObjectInspector, + ObjectInspectorCopyOption.JAVA) + .asInstanceOf[StructObjectInspector] + + val columnTypeNames = structOI + .getAllStructFieldRefs + .map(_.getFieldObjectInspector) + .map(TypeInfoUtils.getTypeInfoFromObjectInspector(_).getTypeName) + .mkString(",") + + hiveConf.set(serdeConstants.LIST_COLUMN_TYPES, columnTypeNames) + hiveConf.set(serdeConstants.LIST_COLUMNS, columnInternalNames) + } + + addColumnMetadataToConf(sc.hiveconf) + @transient def inputRdd = if (!relation.hiveQlTable.isPartitioned) { hadoopReader.makeRDDForTable(relation.hiveQlTable) @@ -143,20 +186,42 @@ case class HiveTableScan( } def execute() = { - inputRdd.map { row => - val values = row match { - case Array(deserializedRow: AnyRef, partitionKeys: Array[String]) => - attributeFunctions.map(_(deserializedRow, partitionKeys)) - case deserializedRow: AnyRef => - attributeFunctions.map(_(deserializedRow, Array.empty)) + inputRdd.mapPartitions { iterator => + if (iterator.isEmpty) { + Iterator.empty + } else { + val mutableRow = new GenericMutableRow(attributes.length) + val mutablePair = new MutablePair[Any, Array[String]]() + val buffered = iterator.buffered + + // NOTE (lian): Critical path of Hive table scan, unnecessary FP style code and pattern + // matching are avoided intentionally. + val rowsAndPartitionKeys = buffered.head match { + // With partition keys + case _: Array[Any] => + buffered.map { case array: Array[Any] => + val deserializedRow = array(0) + val partitionKeys = array(1).asInstanceOf[Array[String]] + mutablePair.update(deserializedRow, partitionKeys) + } + + // Without partition keys + case _ => + val emptyPartitionKeys = Array.empty[String] + buffered.map { deserializedRow => + mutablePair.update(deserializedRow, emptyPartitionKeys) + } + } + + rowsAndPartitionKeys.map { pair => + var i = 0 + while (i < attributes.length) { + mutableRow(i) = attributeFunctions(i)(pair._1, pair._2) + i += 1 + } + mutableRow: Row + } } - buildRow(values.map { - case n: String if n.toLowerCase == "null" => null - case varchar: org.apache.hadoop.hive.common.`type`.HiveVarchar => varchar.getValue - case decimal: org.apache.hadoop.hive.common.`type`.HiveDecimal => - BigDecimal(decimal.bigDecimalValue) - case other => other - }) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index edff38b901073..1b5a132f9665d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -19,11 +19,12 @@ package org.apache.spark.sql.hive.execution import java.io._ +import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} + import org.apache.spark.sql.Logging -import org.apache.spark.sql.catalyst.plans.logical.{ExplainCommand, NativeCommand} +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.Sort -import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} import org.apache.spark.sql.hive.test.TestHive /** @@ -128,17 +129,19 @@ abstract class HiveComparisonTest protected def prepareAnswer( hiveQuery: TestHive.type#HiveQLQueryExecution, answer: Seq[String]): Seq[String] = { + + def isSorted(plan: LogicalPlan): Boolean = plan match { + case _: Join | _: Aggregate | _: BaseRelation | _: Generate | _: Sample | _: Distinct => false + case PhysicalOperation(_, _, Sort(_, _)) => true + case _ => plan.children.iterator.map(isSorted).exists(_ == true) + } + val orderedAnswer = hiveQuery.logical match { // Clean out non-deterministic time schema info. case _: NativeCommand => answer.filterNot(nonDeterministicLine).filterNot(_ == "") case _: ExplainCommand => answer - case _ => - // TODO: Really we only care about the final total ordering here... - val isOrdered = hiveQuery.executedPlan.collect { - case s @ Sort(_, global, _) if global => s - }.nonEmpty - // If the query results aren't sorted, then sort them to ensure deterministic answers. - if (!isOrdered) answer.sorted else answer + case plan if isSorted(plan) => answer + case _ => answer.sorted } orderedAnswer.map(cleanPaths) } @@ -161,7 +164,7 @@ abstract class HiveComparisonTest "minFileSize" ) protected def nonDeterministicLine(line: String) = - nonDeterministicLineIndicators.map(line contains _).reduceLeft(_||_) + nonDeterministicLineIndicators.exists(line contains _) /** * Removes non-deterministic paths from `str` so cached answers will compare correctly. From b7e28fa451511b3b0f849c3d2919ac9c2e4231a1 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 29 May 2014 15:39:25 -0700 Subject: [PATCH 449/641] initial version of LPA A straightforward implementation of LPA algorithm for detecting graph communities using the Pregel framework. Amongst the growing literature on community detection algorithms in networks, LPA is perhaps the most elementary, and despite its flaws it remains a nice and simple approach. Author: Ankur Dave Author: haroldsultan Author: Harold Sultan Closes #905 from haroldsultan/master and squashes the following commits: 327aee0 [haroldsultan] Merge pull request #2 from ankurdave/label-propagation 227a4d0 [Ankur Dave] Untabify 0ac574c [haroldsultan] Merge pull request #1 from ankurdave/label-propagation 0e24303 [Ankur Dave] Add LabelPropagationSuite 84aa061 [Ankur Dave] LabelPropagation: Fix compile errors and style; rename from LPA 9830342 [Harold Sultan] initial version of LPA --- .../spark/graphx/lib/LabelPropagation.scala | 66 +++++++++++++++++++ .../graphx/lib/LabelPropagationSuite.scala | 45 +++++++++++++ 2 files changed, 111 insertions(+) create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/lib/LabelPropagationSuite.scala diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala new file mode 100644 index 0000000000000..776bfb8dd6bfa --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx.lib + +import scala.reflect.ClassTag +import org.apache.spark.graphx._ + +/** Label Propagation algorithm. */ +object LabelPropagation { + /** + * Run static Label Propagation for detecting communities in networks. + * + * Each node in the network is initially assigned to its own community. At every superstep, nodes + * send their community affiliation to all neighbors and update their state to the mode community + * affiliation of incoming messages. + * + * LPA is a standard community detection algorithm for graphs. It is very inexpensive + * computationally, although (1) convergence is not guaranteed and (2) one can end up with + * trivial solutions (all nodes are identified into a single community). + * + * @tparam ED the edge attribute type (not used in the computation) + * + * @param graph the graph for which to compute the community affiliation + * @param maxSteps the number of supersteps of LPA to be performed. Because this is a static + * implementation, the algorithm will run for exactly this many supersteps. + * + * @return a graph with vertex attributes containing the label of community affiliation + */ + def run[ED: ClassTag](graph: Graph[_, ED], maxSteps: Int): Graph[VertexId, ED] = { + val lpaGraph = graph.mapVertices { case (vid, _) => vid } + def sendMessage(e: EdgeTriplet[VertexId, ED]) = { + Iterator((e.srcId, Map(e.dstAttr -> 1L)), (e.dstId, Map(e.srcAttr -> 1L))) + } + def mergeMessage(count1: Map[VertexId, Long], count2: Map[VertexId, Long]) + : Map[VertexId, Long] = { + (count1.keySet ++ count2.keySet).map { i => + val count1Val = count1.getOrElse(i, 0L) + val count2Val = count2.getOrElse(i, 0L) + i -> (count1Val + count2Val) + }.toMap + } + def vertexProgram(vid: VertexId, attr: Long, message: Map[VertexId, Long]) = { + if (message.isEmpty) attr else message.maxBy(_._2)._1 + } + val initialMessage = Map[VertexId, Long]() + Pregel(lpaGraph, initialMessage, maxIterations = maxSteps)( + vprog = vertexProgram, + sendMsg = sendMessage, + mergeMsg = mergeMessage) + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/LabelPropagationSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/LabelPropagationSuite.scala new file mode 100644 index 0000000000000..61fd0c4605568 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/LabelPropagationSuite.scala @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx.lib + +import org.scalatest.FunSuite + +import org.apache.spark.graphx._ + +class LabelPropagationSuite extends FunSuite with LocalSparkContext { + test("Label Propagation") { + withSpark { sc => + // Construct a graph with two cliques connected by a single edge + val n = 5 + val clique1 = for (u <- 0L until n; v <- 0L until n) yield Edge(u, v, 1) + val clique2 = for (u <- 0L to n; v <- 0L to n) yield Edge(u + n, v + n, 1) + val twoCliques = sc.parallelize(clique1 ++ clique2 :+ Edge(0L, n, 1)) + val graph = Graph.fromEdges(twoCliques, 1) + // Run label propagation + val labels = LabelPropagation.run(graph, n * 4).cache() + + // All vertices within a clique should have the same label + val clique1Labels = labels.vertices.filter(_._1 < n).map(_._2).collect.toArray + assert(clique1Labels.forall(_ == clique1Labels(0))) + val clique2Labels = labels.vertices.filter(_._1 >= n).map(_._2).collect.toArray + assert(clique2Labels.forall(_ == clique2Labels(0))) + // The two cliques should have different labels + assert(clique1Labels(0) != clique2Labels(0)) + } + } +} From eeee978a348ec2a35cc27865cea6357f9db75b74 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 29 May 2014 23:20:20 -0700 Subject: [PATCH 450/641] [SPARK-1820] Make GenerateMimaIgnore @DeveloperApi annotation aware. We add all the classes annotated as `DeveloperApi` to `~/.mima-excludes`. Author: Prashant Sharma Author: nikhil7sh Closes #904 from ScrapCodes/SPARK-1820/ignore-Developer-Api and squashes the following commits: de944f9 [Prashant Sharma] Code review. e3c5215 [Prashant Sharma] Incorporated patrick's suggestions and fixed the scalastyle build. 9983a42 [nikhil7sh] [SPARK-1820] Make GenerateMimaIgnore @DeveloperApi annotation aware --- .../spark/tools/GenerateMIMAIgnore.scala | 22 ++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala index a433e8e2e89f4..011db50b7d568 100644 --- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala +++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala @@ -23,6 +23,7 @@ import java.util.jar.JarFile import scala.collection.mutable import scala.collection.JavaConversions._ import scala.reflect.runtime.universe.runtimeMirror +import scala.reflect.runtime.{universe => unv} /** * A tool for generating classes to be excluded during binary checking with MIMA. It is expected @@ -42,7 +43,7 @@ object GenerateMIMAIgnore { private def classesPrivateWithin(packageName: String): Set[String] = { val classes = getClasses(packageName) - val privateClasses = mutable.HashSet[String]() + val ignoredClasses = mutable.HashSet[String]() def isPackagePrivate(className: String) = { try { @@ -70,8 +71,21 @@ object GenerateMIMAIgnore { } } + def isDeveloperApi(className: String) = { + try { + val clazz = mirror.classSymbol(Class.forName(className, false, classLoader)) + clazz.annotations.exists(_.tpe =:= unv.typeOf[org.apache.spark.annotation.DeveloperApi]) + } catch { + case _: Throwable => { + println("Error determining Annotations: " + className) + false + } + } + } + for (className <- classes) { val directlyPrivateSpark = isPackagePrivate(className) + val developerApi = isDeveloperApi(className) /* Inner classes defined within a private[spark] class or object are effectively invisible, so we account for them as package private. */ @@ -83,9 +97,11 @@ object GenerateMIMAIgnore { false } } - if (directlyPrivateSpark || indirectlyPrivateSpark) privateClasses += className + if (directlyPrivateSpark || indirectlyPrivateSpark || developerApi) { + ignoredClasses += className + } } - privateClasses.flatMap(c => Seq(c, c.replace("$", "#"))).toSet + ignoredClasses.flatMap(c => Seq(c, c.replace("$", "#"))).toSet } def main(args: Array[String]) { From c8bf4131bc2a2e147e977159fc90e94b85738830 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 30 May 2014 00:34:33 -0700 Subject: [PATCH 451/641] [SPARK-1566] consolidate programming guide, and general doc updates This is a fairly large PR to clean up and update the docs for 1.0. The major changes are: * A unified programming guide for all languages replaces language-specific ones and shows language-specific info in tabs * New programming guide sections on key-value pairs, unit testing, input formats beyond text, migrating from 0.9, and passing functions to Spark * Spark-submit guide moved to a separate page and expanded slightly * Various cleanups of the menu system, security docs, and others * Updated look of title bar to differentiate the docs from previous Spark versions You can find the updated docs at http://people.apache.org/~matei/1.0-docs/_site/ and in particular http://people.apache.org/~matei/1.0-docs/_site/programming-guide.html. Author: Matei Zaharia Closes #896 from mateiz/1.0-docs and squashes the following commits: 03e6853 [Matei Zaharia] Some tweaks to configuration and YARN docs 0779508 [Matei Zaharia] tweak ef671d4 [Matei Zaharia] Keep frames in JavaDoc links, and other small tweaks 1bf4112 [Matei Zaharia] Review comments 4414f88 [Matei Zaharia] tweaks d04e979 [Matei Zaharia] Fix some old links to Java guide a34ed33 [Matei Zaharia] tweak 541bb3b [Matei Zaharia] miscellaneous changes fcefdec [Matei Zaharia] Moved submitting apps to separate doc 61d72b4 [Matei Zaharia] stuff 181f217 [Matei Zaharia] migration guide, remove old language guides e11a0da [Matei Zaharia] Add more API functions 6a030a9 [Matei Zaharia] tweaks 8db0ae3 [Matei Zaharia] Added key-value pairs section 318d2c9 [Matei Zaharia] tweaks 1c81477 [Matei Zaharia] New section on basics and function syntax e38f559 [Matei Zaharia] Actually added programming guide to Git a33d6fe [Matei Zaharia] First pass at updating programming guide to support all languages, plus other tweaks throughout 3b6a876 [Matei Zaharia] More CSS tweaks 01ec8bf [Matei Zaharia] More CSS tweaks e6d252e [Matei Zaharia] Change color of doc title bar to differentiate from 0.9.0 --- docs/_layouts/global.html | 18 +- docs/bagel-programming-guide.md | 2 +- docs/building-with-maven.md | 90 +- docs/cluster-overview.md | 108 +- docs/configuration.md | 11 +- docs/css/bootstrap.min.css | 2 +- docs/graphx-programming-guide.md | 8 +- docs/hadoop-third-party-distributions.md | 2 +- docs/index.md | 79 +- docs/java-programming-guide.md | 215 +--- docs/js/api-docs.js | 23 +- docs/js/main.js | 21 + docs/mllib-guide.md | 10 +- docs/mllib-optimization.md | 2 +- docs/monitoring.md | 2 +- docs/programming-guide.md | 1294 ++++++++++++++++++++++ docs/python-programming-guide.md | 168 +-- docs/quick-start.md | 39 +- docs/running-on-mesos.md | 7 +- docs/running-on-yarn.md | 91 +- docs/scala-programming-guide.md | 445 +------- docs/security.md | 18 +- docs/spark-standalone.md | 4 +- docs/sql-programming-guide.md | 29 +- docs/streaming-programming-guide.md | 42 +- docs/submitting-applications.md | 153 +++ docs/tuning.md | 6 +- 27 files changed, 1767 insertions(+), 1122 deletions(-) create mode 100644 docs/programming-guide.md create mode 100644 docs/submitting-applications.md diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index fb808129bb65d..4ba20e590f2c2 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -9,6 +9,11 @@ {{ page.title }} - Spark {{site.SPARK_VERSION_SHORT}} Documentation + {% if page.redirect %} + + + {% endif %} +