From d000ca98a80986ff5b13477547f1dcab7856ae63 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 2 Jun 2014 12:09:43 -0700 Subject: [PATCH 01/54] [SPARK-1958] Calling .collect() on a SchemaRDD should call executeCollect() on the underlying query plan. In cases like `Limit` and `TakeOrdered`, `executeCollect()` makes optimizations that `execute().collect()` will not. Author: Cheng Lian Closes #939 from liancheng/spark-1958 and squashes the following commits: bdc4a14 [Cheng Lian] Copy rows to present immutable data to users 8250976 [Cheng Lian] Added return type explicitly for public API 192a25c [Cheng Lian] [SPARK-1958] Calling .collect() on a SchemaRDD should call executeCollect() on the underlying query plan. --- .../src/main/scala/org/apache/spark/sql/SchemaRDD.scala | 6 ++++++ .../scala/org/apache/spark/sql/execution/SparkPlan.scala | 2 +- .../org/apache/spark/sql/parquet/ParquetQuerySuite.scala | 2 +- 3 files changed, 8 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 e855f36256bc5..8855c4e876917 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 @@ -368,6 +368,12 @@ class SchemaRDD( new SchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(logicalPlan.output, rdd))) } + // ======================================================================= + // Overriden RDD actions + // ======================================================================= + + override def collect(): Array[Row] = queryExecution.executedPlan.executeCollect() + // ======================================================================= // Base RDD functions that do NOT change schema // ======================================================================= 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 235a9b1692460..4613df103943d 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 @@ -49,7 +49,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { /** * Runs this query returning the result as an array. */ - def executeCollect(): Array[Row] = execute().collect() + def executeCollect(): Array[Row] = execute().map(_.copy()).collect() protected def buildRow(values: Seq[Any]): Row = new GenericRow(values.toArray) 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 f9731e82e4924..b973ceba5fec0 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 @@ -201,7 +201,7 @@ class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { } test("insert (appending) to same table via Scala API") { - sql("INSERT INTO testsource SELECT * FROM testsource").collect() + sql("INSERT INTO testsource SELECT * FROM testsource") val double_rdd = sql("SELECT * FROM testsource").collect() assert(double_rdd != null) assert(double_rdd.size === 30) From ec8be274a7bc586bb5b025033cbfd73f9a4d7160 Mon Sep 17 00:00:00 2001 From: egraldlo Date: Mon, 2 Jun 2014 18:02:57 -0700 Subject: [PATCH 02/54] [SPARK-1995][SQL] system function upper and lower can be supported I don't know whether it's time to implement system function about string operation in spark sql now. Author: egraldlo Closes #936 from egraldlo/stringoperator and squashes the following commits: 3c6c60a [egraldlo] Add UPPER, LOWER, MAX and MIN into hive parser ea76d0a [egraldlo] modify the formatting issues b49f25e [egraldlo] modify the formatting issues 1f0bbb5 [egraldlo] system function upper and lower supported 13d3267 [egraldlo] system function upper and lower supported --- .../apache/spark/sql/catalyst/SqlParser.scala | 4 +++ .../expressions/stringOperations.scala | 32 +++++++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 23 +++++++++++++ .../org/apache/spark/sql/hive/HiveQl.scala | 12 ++++++- 4 files changed, 70 insertions(+), 1 deletion(-) 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 f2b9b2c1a3ad5..a404e7441a1bd 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 @@ -124,6 +124,8 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val OVERWRITE = Keyword("OVERWRITE") protected val LIKE = Keyword("LIKE") protected val RLIKE = Keyword("RLIKE") + protected val UPPER = Keyword("UPPER") + protected val LOWER = Keyword("LOWER") protected val REGEXP = Keyword("REGEXP") protected val ORDER = Keyword("ORDER") protected val OUTER = Keyword("OUTER") @@ -329,6 +331,8 @@ class SqlParser extends StandardTokenParsers with PackratParsers { AVG ~> "(" ~> expression <~ ")" ^^ { case exp => Average(exp) } | MIN ~> "(" ~> expression <~ ")" ^^ { case exp => Min(exp) } | MAX ~> "(" ~> expression <~ ")" ^^ { case exp => Max(exp) } | + UPPER ~> "(" ~> expression <~ ")" ^^ { case exp => Upper(exp) } | + LOWER ~> "(" ~> expression <~ ")" ^^ { case exp => Lower(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/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index ddc16ce87b895..dcded0774180e 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 @@ -70,6 +70,22 @@ trait StringRegexExpression { } } +trait CaseConversionExpression { + self: UnaryExpression => + + type EvaluatedType = Any + + def convert(v: String): String + + def nullable: Boolean = child.nullable + def dataType: DataType = StringType + + override def eval(input: Row): Any = { + val converted = child.eval(input) + convert(converted.toString) + } +} + /** * Simple RegEx pattern matching function */ @@ -115,3 +131,19 @@ case class RLike(left: Expression, right: Expression) override def escape(v: String): String = v override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) } + +/** + * A function that converts the characters of a string to uppercase. + */ +case class Upper(child: Expression) extends UnaryExpression with CaseConversionExpression { + + override def convert(v: String): String = v.toUpperCase() +} + +/** + * A function that converts the characters of a string to lowercase. + */ +case class Lower(child: Expression) extends UnaryExpression with CaseConversionExpression { + + override def convert(v: String): String = v.toLowerCase() +} 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 189dccd5253e5..95860e6683f67 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 @@ -313,4 +313,27 @@ class SQLQuerySuite extends QueryTest { (3, "C"), (4, "D"))) } + + test("system function upper()") { + checkAnswer( + sql("SELECT n,UPPER(l) FROM lowerCaseData"), + Seq( + (1, "A"), + (2, "B"), + (3, "C"), + (4, "D"))) + } + + test("system function lower()") { + checkAnswer( + sql("SELECT N,LOWER(L) FROM upperCaseData"), + Seq( + (1, "a"), + (2, "b"), + (3, "c"), + (4, "d"), + (5, "e"), + (6, "f"))) + } + } 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 93b9057a23816..e8a3ee5535b6e 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 @@ -781,6 +781,10 @@ private[hive] object HiveQl { val COUNT = "(?i)COUNT".r val AVG = "(?i)AVG".r val SUM = "(?i)SUM".r + val MAX = "(?i)MAX".r + val MIN = "(?i)MIN".r + val UPPER = "(?i)UPPER".r + val LOWER = "(?i)LOWER".r val RAND = "(?i)RAND".r val AND = "(?i)AND".r val OR = "(?i)OR".r @@ -817,7 +821,13 @@ private[hive] object HiveQl { case Token("TOK_FUNCTIONDI", Token(COUNT(), Nil) :: args) => CountDistinct(args.map(nodeToExpr)) case Token("TOK_FUNCTION", Token(SUM(), Nil) :: arg :: Nil) => Sum(nodeToExpr(arg)) case Token("TOK_FUNCTIONDI", Token(SUM(), Nil) :: arg :: Nil) => SumDistinct(nodeToExpr(arg)) - + case Token("TOK_FUNCTION", Token(MAX(), Nil) :: arg :: Nil) => Max(nodeToExpr(arg)) + case Token("TOK_FUNCTION", Token(MIN(), Nil) :: arg :: Nil) => Min(nodeToExpr(arg)) + + /* System functions about string operations */ + case Token("TOK_FUNCTION", Token(UPPER(), Nil) :: arg :: Nil) => Upper(nodeToExpr(arg)) + case Token("TOK_FUNCTION", Token(LOWER(), Nil) :: arg :: Nil) => Lower(nodeToExpr(arg)) + /* Casts */ case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) From 862283e9ccace6824880aa4e161723fb3248d438 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 2 Jun 2014 19:20:23 -0700 Subject: [PATCH 03/54] Avoid dynamic dispatching when unwrapping Hive data. This is a follow up of PR #758. The `unwrapHiveData` function is now composed statically before actual rows are scanned according to the field object inspector to avoid dynamic dispatching cost. According to the same micro benchmark used in PR #758, this simple change brings slight performance boost: 2.5% for CSV table and 1% for RCFile table. ``` Optimized version: CSV: 6870 ms, RCFile: 5687 ms CSV: 6832 ms, RCFile: 5800 ms CSV: 6822 ms, RCFile: 5679 ms CSV: 6704 ms, RCFile: 5758 ms CSV: 6819 ms, RCFile: 5725 ms Original version: CSV: 7042 ms, RCFile: 5667 ms CSV: 6883 ms, RCFile: 5703 ms CSV: 7115 ms, RCFile: 5665 ms CSV: 7020 ms, RCFile: 5981 ms CSV: 6871 ms, RCFile: 5906 ms ``` Author: Cheng Lian Closes #935 from liancheng/staticUnwrapping and squashes the following commits: c49c70c [Cheng Lian] Avoid dynamic dispatching when unwrapping Hive data. --- .../apache/spark/sql/hive/hiveOperators.scala | 28 +++++++++++-------- .../hive/execution/HiveComparisonTest.scala | 5 ++-- 2 files changed, 18 insertions(+), 15 deletions(-) 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 d263c312960a9..29b4b9b006e45 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 @@ -26,8 +26,7 @@ import org.apache.hadoop.hive.ql.plan.{TableDesc, FileSinkDesc} 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.objectinspector.primitive._ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Serializer} import org.apache.hadoop.io.Writable @@ -95,29 +94,34 @@ case class HiveTableScan( attributes.map { a => val ordinal = relation.partitionKeys.indexOf(a) if (ordinal >= 0) { + val dataType = relation.partitionKeys(ordinal).dataType (_: Any, partitionKeys: Array[String]) => { - val value = partitionKeys(ordinal) - val dataType = relation.partitionKeys(ordinal).dataType - unwrapHiveData(castFromString(value, dataType)) + castFromString(partitionKeys(ordinal), dataType) } } else { val ref = objectInspector.getAllStructFieldRefs .find(_.getFieldName == a.name) .getOrElse(sys.error(s"Can't find attribute $a")) + val fieldObjectInspector = ref.getFieldObjectInspector + + val unwrapHiveData = fieldObjectInspector match { + case _: HiveVarcharObjectInspector => + (value: Any) => value.asInstanceOf[HiveVarchar].getValue + case _: HiveDecimalObjectInspector => + (value: Any) => BigDecimal(value.asInstanceOf[HiveDecimal].bigDecimalValue()) + case _ => + identity[Any] _ + } + (row: Any, _: Array[String]) => { val data = objectInspector.getStructFieldData(row, ref) - unwrapHiveData(unwrapData(data, ref.getFieldObjectInspector)) + val hiveData = unwrapData(data, fieldObjectInspector) + if (hiveData != null) unwrapHiveData(hiveData) else null } } } } - private def unwrapHiveData(value: Any) = value match { - 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) } 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 1b5a132f9665d..0f954103a85f2 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 @@ -133,15 +133,14 @@ abstract class HiveComparisonTest 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) + case _ => plan.children.iterator.exists(isSorted) } val orderedAnswer = hiveQuery.logical match { // Clean out non-deterministic time schema info. case _: NativeCommand => answer.filterNot(nonDeterministicLine).filterNot(_ == "") case _: ExplainCommand => answer - case plan if isSorted(plan) => answer - case _ => answer.sorted + case plan => if (isSorted(plan)) answer else answer.sorted } orderedAnswer.map(cleanPaths) } From 7782a304ad105ec95cf62cb799e365e5fb385a69 Mon Sep 17 00:00:00 2001 From: Syed Hashmi Date: Tue, 3 Jun 2014 12:04:47 -0700 Subject: [PATCH 04/54] [SPARK-1942] Stop clearing spark.driver.port in unit tests stop resetting spark.driver.port in unit tests (scala, java and python). Author: Syed Hashmi Author: CodingCat Closes #943 from syedhashmi/master and squashes the following commits: 885f210 [Syed Hashmi] Removing unnecessary file (created by mergetool) b8bd4b5 [Syed Hashmi] Merge remote-tracking branch 'upstream/master' b895e59 [Syed Hashmi] Revert "[SPARK-1784] Add a new partitioner" 57b6587 [Syed Hashmi] Revert "[SPARK-1784] Add a balanced partitioner" 1574769 [Syed Hashmi] [SPARK-1942] Stop clearing spark.driver.port in unit tests 4354836 [Syed Hashmi] Revert "SPARK-1686: keep schedule() calling in the main thread" fd36542 [Syed Hashmi] [SPARK-1784] Add a balanced partitioner 6668015 [CodingCat] SPARK-1686: keep schedule() calling in the main thread 4ca94cc [Syed Hashmi] [SPARK-1784] Add a new partitioner --- bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala | 2 -- core/src/test/java/org/apache/spark/JavaAPISuite.java | 3 --- core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala | 4 ---- .../test/scala/org/apache/spark/MapOutputTrackerSuite.scala | 3 --- .../scala/org/apache/spark/storage/BlockManagerSuite.scala | 2 -- .../src/test/java/org/apache/spark/Java8APISuite.java | 2 -- .../scala/org/apache/spark/graphx/LocalSparkContext.scala | 2 -- .../mllib/classification/JavaLogisticRegressionSuite.java | 1 - .../spark/mllib/classification/JavaNaiveBayesSuite.java | 1 - .../org/apache/spark/mllib/classification/JavaSVMSuite.java | 1 - .../org/apache/spark/mllib/clustering/JavaKMeansSuite.java | 1 - .../org/apache/spark/mllib/recommendation/JavaALSSuite.java | 1 - .../org/apache/spark/mllib/regression/JavaLassoSuite.java | 1 - .../spark/mllib/regression/JavaLinearRegressionSuite.java | 1 - .../spark/mllib/regression/JavaRidgeRegressionSuite.java | 1 - .../scala/org/apache/spark/mllib/util/LocalSparkContext.scala | 1 - python/pyspark/tests.py | 4 ---- repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala | 2 -- .../src/main/scala/org/apache/spark/sql/hive/TestHive.scala | 1 - .../org/apache/spark/streaming/util/MasterFailureTest.scala | 1 - .../org/apache/spark/streaming/LocalJavaStreamingContext.java | 4 ---- .../scala/org/apache/spark/streaming/CheckpointSuite.scala | 1 - .../test/scala/org/apache/spark/streaming/TestSuiteBase.scala | 2 -- 23 files changed, 42 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 8e0f82ddb8897..110bd0a9a0c41 100644 --- a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala @@ -38,8 +38,6 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeo sc.stop() sc = null } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port") } test("halting by voting") { diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 3dd79243ab5bd..7193223addf66 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -68,9 +68,6 @@ public void setUp() { public void tearDown() { sc.stop(); 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 { diff --git a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala index c645e4cbe8132..4ab870e751778 100644 --- a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala @@ -39,7 +39,6 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val hostname = "localhost" 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) assert(securityManager.isAuthenticationEnabled() === true) @@ -77,7 +76,6 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val hostname = "localhost" 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) assert(securityManager.isAuthenticationEnabled() === false) @@ -129,7 +127,6 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val hostname = "localhost" 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) assert(securityManager.isAuthenticationEnabled() === true) @@ -182,7 +179,6 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val hostname = "localhost" 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) assert(securityManager.isAuthenticationEnabled() === true) diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 6b2571cd9295e..95ba273f16a71 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -124,9 +124,6 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = new SecurityManager(conf)) - // Will be cleared by LocalSparkContext - System.setProperty("spark.driver.port", boundPort.toString) - val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") 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 00deecc1c3ca9..81bd8257bc155 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -78,8 +78,6 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } after { - System.clearProperty("spark.driver.port") - if (store != null) { store.stop() store = null 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 84d3b6f243c72..c366c10b15a20 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 @@ -58,8 +58,6 @@ public void setUp() { public void tearDown() { sc.stop(); sc = null; - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port"); } @Test diff --git a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala index 51f02f94e00d5..47594a800a3b1 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala @@ -38,8 +38,6 @@ trait LocalSparkContext { f(sc) } finally { sc.stop() - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port") } } } 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 d75d3a6b26730..faa675b59cd50 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 @@ -42,7 +42,6 @@ public void setUp() { public void tearDown() { sc.stop(); sc = null; - System.clearProperty("spark.driver.port"); } int validatePrediction(List validationData, LogisticRegressionModel model) { 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 743a43a139c0c..1c90522a0714a 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 @@ -44,7 +44,6 @@ public void setUp() { public void tearDown() { sc.stop(); sc = null; - System.clearProperty("spark.driver.port"); } private static final List POINTS = Arrays.asList( 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 667f76a1bd55f..31b9f3e8d438e 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 @@ -41,7 +41,6 @@ public void setUp() { public void tearDown() { sc.stop(); sc = null; - System.clearProperty("spark.driver.port"); } int validatePrediction(List validationData, SVMModel model) { 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 0c916ca378034..31676e64025d0 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 @@ -44,7 +44,6 @@ public void setUp() { public void tearDown() { sc.stop(); sc = null; - System.clearProperty("spark.driver.port"); } @Test diff --git a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java index b150334deb06c..bf2365f82044c 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java @@ -42,7 +42,6 @@ public void setUp() { public void tearDown() { sc.stop(); sc = null; - System.clearProperty("spark.driver.port"); } static void validatePrediction(MatrixFactorizationModel model, int users, int products, int features, diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java index f725924a2d971..8950b48888b74 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java @@ -41,7 +41,6 @@ public void setUp() { public void tearDown() { sc.stop(); sc = null; - System.clearProperty("spark.driver.port"); } int validatePrediction(List validationData, LassoModel model) { 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 6dc6877691036..24c4c20d9af18 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 @@ -43,7 +43,6 @@ public void setUp() { public void tearDown() { sc.stop(); sc = null; - System.clearProperty("spark.driver.port"); } int validatePrediction(List validationData, LinearRegressionModel model) { diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java index 03714ae7e4d00..7266eec235800 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java @@ -43,7 +43,6 @@ public void setUp() { public void tearDown() { sc.stop(); sc = null; - System.clearProperty("spark.driver.port"); } double predictionError(List validationData, RidgeRegressionModel model) { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala index 212fbe9288f0d..0d4868f3d9e42 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala @@ -34,7 +34,6 @@ trait LocalSparkContext extends BeforeAndAfterAll { self: Suite => if (sc != null) { sc.stop() } - System.clearProperty("spark.driver.port") super.afterAll() } } diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index ed90915fcda35..1f2a6ea941cf2 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -57,10 +57,6 @@ def setUp(self): def tearDown(self): self.sc.stop() sys.path = self._old_sys_path - # To avoid Akka rebinding to the same port, since it doesn't unbind - # immediately on shutdown - self.sc._jvm.System.clearProperty("spark.driver.port") - class TestCheckpoint(PySparkTestCase): 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 95460aa205331..98cdfd0054713 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -51,8 +51,6 @@ class ReplSuite extends FunSuite { if (interp.sparkContext != null) { interp.sparkContext.stop() } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port") return out.toString } 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 fa7d010459c63..041e813598d1b 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 @@ -58,7 +58,6 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { // By clearing the port we force Spark to pick a new one. This allows us to rerun tests // without restarting the JVM. - System.clearProperty("spark.driver.port") System.clearProperty("spark.hostPort") override lazy val warehousePath = getTempFilePath("sparkHiveWarehouse").getCanonicalPath 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 b3ed302db6a38..98e17ff92e205 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 @@ -187,7 +187,6 @@ object MasterFailureTest extends Logging { setupCalled = true // Setup the streaming computation with the given operation - System.clearProperty("spark.driver.port") val ssc = new StreamingContext("local[4]", "MasterFailureTest", batchDuration, null, Nil, Map()) ssc.checkpoint(checkpointDir.toString) diff --git a/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 849bbf1299182..6e1f01900071b 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -27,7 +27,6 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.clearProperty("spark.driver.port"); System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); ssc.checkpoint("checkpoint"); @@ -37,8 +36,5 @@ public void setUp() { public void tearDown() { ssc.stop(); ssc = null; - - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port"); } } 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 d20a7b728c741..10ad3c9e1adc9 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -370,7 +370,6 @@ class CheckpointSuite extends TestSuiteBase { "\n-------------------------------------------\n" ) ssc = new StreamingContext(checkpointDir) - System.clearProperty("spark.driver.port") ssc.start() val outputNew = advanceTimeWithRealDelay[V](ssc, nextNumBatches) // the first element will be re-processed data of the last batch before restart 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 8036f77c973ae..cc178fba12c9d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -153,8 +153,6 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Default after function for any streaming test suite. Override this // if you want to add your stuff to "after" (i.e., don't call after { } ) 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") } From 6c044ed100081f1fa7c8df4bd4b58b65a61c3360 Mon Sep 17 00:00:00 2001 From: Henry Saputra Date: Tue, 3 Jun 2014 13:03:51 -0700 Subject: [PATCH 05/54] SPARK-2001 : Remove docs/spark-debugger.md from master Per discussion in dev list: " Seemed like the spark-debugger.md is no longer accurate (see http://spark.apache.org/docs/latest/spark-debugger.html) and since it was originally written Spark has evolved that makes the doc obsolete. There are already work pending for new replay debugging (I could not find the PR links for it) so I With version control we could always reinstate the old doc if needed, but as of today the doc is no longer reflect the current state of Spark's RDD. " Author: Henry Saputra Closes #953 from hsaputra/SPARK-2001-hsaputra and squashes the following commits: dc324aa [Henry Saputra] SPARK-2001 : Remove docs/spark-debugger.md from master since it is obsolete --- docs/spark-debugger.md | 121 ----------------------------------------- 1 file changed, 121 deletions(-) delete mode 100644 docs/spark-debugger.md diff --git a/docs/spark-debugger.md b/docs/spark-debugger.md deleted file mode 100644 index 35d06c51aaf0e..0000000000000 --- a/docs/spark-debugger.md +++ /dev/null @@ -1,121 +0,0 @@ ---- -layout: global -title: The Spark Debugger ---- -**Summary:** The Spark debugger provides replay debugging for deterministic (logic) errors in Spark programs. It's currently in development, but you can try it out in the [arthur branch](https://github.com/apache/spark/tree/arthur). - -## Introduction - -From a user's point of view, debugging a general distributed program can be tedious and confusing. Many distributed programs are nondeterministic; their outcome depends on the interleaving between computation and message passing across multiple machines. Also, the fact that a program is running on a cluster of hundreds or thousands of machines means that it's hard to understand the program state and pinpoint the location of problems. - -In order to tame nondeterminism, a distributed debugger has to log a lot of information, imposing a serious performance penalty on the application being debugged. - -But the Spark programming model lets us provide replay debugging for almost zero overhead. Spark programs are a series of RDDs and deterministic transformations, so when debugging a Spark program, we don't have to debug it all at once -- instead, we can debug each transformation individually. Broadly, the debugger lets us do the following two things: - -* Recompute and inspect intermediate RDDs after the program has finished. -* Re-run a particular task in a single-threaded debugger to find exactly what went wrong. - -For deterministic errors, debugging a Spark program is now as easy as debugging a single-threaded one. - -## Approach - -As your Spark program runs, the slaves report key events back to the master -- for example, RDD creations, RDD contents, and uncaught exceptions. (A full list of event types is in [EventLogging.scala](https://github.com/apache/spark/blob/arthur/core/src/main/scala/spark/EventLogging.scala).) The master logs those events, and you can load the event log into the debugger after your program is done running. - -_A note on nondeterminism:_ For fault recovery, Spark requires RDD transformations (for example, the function passed to `RDD.map`) to be deterministic. The Spark debugger also relies on this property, and it can also warn you if your transformation is nondeterministic. This works by checksumming the contents of each RDD and comparing the checksums from the original execution to the checksums after recomputing the RDD in the debugger. - -## Usage - -### Enabling the event log - -To turn on event logging for your program, set `$SPARK_JAVA_OPTS` in `conf/spark-env.sh` as follows: - -{% highlight bash %} -export SPARK_JAVA_OPTS='-Dspark.arthur.logPath=path/to/event-log' -{% endhighlight %} - -where `path/to/event-log` is where you want the event log to go relative to `$SPARK_HOME`. - -**Warning:** If `path/to/event-log` already exists, event logging will be automatically disabled. - -### Loading the event log into the debugger - -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")) -r: spark.EventLogReader = spark.EventLogReader@726b37ad -{% endhighlight %} - - **Warning:** If the event log doesn't exist or is unreadable, this will silently fail and `r.events` will be empty. - -### Exploring intermediate RDDs - -Use `r.rdds` to get a list of intermediate RDDs generated during your program's execution. An RDD with id _x_ is located at r.rdds(x). For example: - -{% highlight scala %} -scala> r.rdds -res8: scala.collection.mutable.ArrayBuffer[spark.RDD[_]] = ArrayBuffer(spark.HadoopRDD@fe85adf, spark.MappedRDD@5fa5eea1, spark.MappedRDD@6d5bd16, spark.ShuffledRDD@3a70f2db, spark.FlatMappedValuesRDD@4d5825d6, spark.MappedValuesRDD@561c2c45, spark.CoGroupedRDD@539e922d, spark.MappedValuesRDD@4f8ef33e, spark.FlatMappedRDD@32039440, spark.ShuffledRDD@8fa0f67, spark.MappedValuesRDD@590937cb, spark.CoGroupedRDD@6c2e1e17, spark.MappedValuesRDD@47b9af7d, spark.FlatMappedRDD@6fb05c54, spark.ShuffledRDD@237dc815, spark.MappedValuesRDD@16daece7, spark.CoGroupedRDD@7ef73d69, spark.MappedValuesRDD@19e0f99e, spark.FlatMappedRDD@1240158, spark.ShuffledRDD@62d438fd, spark.MappedValuesRDD@5ae99cbb, spark.FilteredRDD@1f30e79e, spark.MappedRDD@43b64611) -{% endhighlight %} - -Use `r.printRDDs()` to get a formatted list of intermediate RDDs, along with the source location where they were created. For example: - -{% highlight scala %} -scala> r.printRDDs -#00: HadoopRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:31) -#01: MappedRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:31) -#02: MappedRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:35) -#03: ShuffledRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:35) -#04: FlatMappedValuesRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:35) -#05: MappedValuesRDD spark.bagel.examples.WikipediaPageRankStandalone$.pageRank(WikipediaPageRankStandalone.scala:91) -#06: CoGroupedRDD spark.bagel.examples.WikipediaPageRankStandalone$.pageRank(WikipediaPageRankStandalone.scala:92) -[...] -{% endhighlight %} - -Use `r.visualizeRDDs()` to visualize the RDDs as a dependency graph. For example: - -{% highlight scala %} -scala> r.visualizeRDDs -/tmp/spark-rdds-3758182885839775712.pdf -{% endhighlight %} - -![Example RDD dependency graph](http://www.ankurdave.com/images/rdd-dep-graph.png) - -Iterate over the `RDDCreation` entries in `r.events` (e.g. `for (RDDCreation(rdd, location) <- events)`) to access the RDD creation locations as well as the RDDs themselves. - -### Debugging a particular task - -1. Find the task you want to debug. If the task threw an exception, the `ExceptionEvent` that was created will have a reference to the task. For example: - {% highlight scala %} -spark> val task = r.events.collect { case e: ExceptionEvent => e }.head.task -{% endhighlight %} - Otherwise, look through the list of all tasks in `r.tasks`, or browse tasks by RDD using r.tasksForRDD(rdd), which returns a list of tasks whose input is the given RDD. - -2. Run the task by calling r.debugTask(taskStageId, taskPartition). The task should contain these two values; you can extract them as follows: - {% highlight scala %} -val (taskStageId, taskPartition) = task match { - case rt: ResultTask[_, _] => (rt.stageId, rt.partition) - case smt: ShuffleMapTask => (smt.stageId, smt.partition) - case _ => throw new UnsupportedOperationException -}) -{% endhighlight %} - The Spark debugger will launch the task in a separate JVM, but you will see the task's stdout and stderr inline with the Spark shell. If you want to pass custom debugging arguments to the task's JVM (for example, to change the debugging port), set the optional `debugOpts` argument to `r.debugTask`. When `debugOpts` is left unset, it defaults to: - {% highlight scala %} --Xdebug -agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=8000 -{% endhighlight %} - -3. In another terminal, attach your favorite conventional debugger to the Spark shell. For example, if you want to use jdb, run `jdb -attach 8000`. - -4. Debug the task as you would debug a normal program. For example, to break when an exception is thrown: - {% highlight scala %} -> catch org.xml.sax.SAXParseException -{% endhighlight %} - -5. When the task ends, its JVM will quit and control will return to the main Spark shell. To stop it prematurely, you can kill it from the debugger, or interrupt it from the terminal with Ctrl-C. - -### Detecting nondeterminism in your transformations - -When a task gets run more than once, Arthur is able to compare the checksums of the task's output. If they are different, Arthur will insert a `ChecksumEvent` into `r.checksumMismatches` and print a warning like the following: - {% highlight scala %} -12/04/07 11:42:44 WARN spark.EventLogWriter: Nondeterminism detected in shuffle output on RDD 2, partition 3, output split 0 -{% endhighlight %} - From 45e9bc85db231e84a23b8d757136023eabcec13e Mon Sep 17 00:00:00 2001 From: "Wenchen Fan(Cloud)" Date: Tue, 3 Jun 2014 13:18:20 -0700 Subject: [PATCH 06/54] [SPARK-1912] fix compress memory issue during reduce When we need to read a compressed block, we will first create a compress stream instance(LZF or Snappy) and use it to wrap that block. Let's say a reducer task need to read 1000 local shuffle blocks, it will first prepare to read that 1000 blocks, which means create 1000 compression stream instance to wrap them. But the initialization of compression instance will allocate some memory and when we have many compression instance at the same time, it is a problem. Actually reducer reads the shuffle blocks one by one, so we can do the compression instance initialization lazily. Author: Wenchen Fan(Cloud) Closes #860 from cloud-fan/fix-compress and squashes the following commits: 0924a6b [Wenchen Fan(Cloud)] rename 'doWork' into 'getIterator' 07f32c2 [Wenchen Fan(Cloud)] move the LazyProxyIterator to dataDeserialize d80c426 [Wenchen Fan(Cloud)] remove empty lines in short class 2c8adb2 [Wenchen Fan(Cloud)] add inline comment 8ebff77 [Wenchen Fan(Cloud)] fix compress memory issue during reduce --- .../apache/spark/storage/BlockManager.scala | 22 +++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) 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 6e450081dcb11..a41286d3e4a00 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1015,8 +1015,26 @@ private[spark] class BlockManager( bytes: ByteBuffer, serializer: Serializer = defaultSerializer): Iterator[Any] = { bytes.rewind() - val stream = wrapForCompression(blockId, new ByteBufferInputStream(bytes, true)) - serializer.newInstance().deserializeStream(stream).asIterator + + def getIterator = { + val stream = wrapForCompression(blockId, new ByteBufferInputStream(bytes, true)) + serializer.newInstance().deserializeStream(stream).asIterator + } + + if (blockId.isShuffle) { + // Reducer may need to read many local shuffle blocks and will wrap them into Iterators + // at the beginning. The wrapping will cost some memory (compression instance + // initialization, etc.). Reducer read shuffle blocks one by one so we could do the + // wrapping lazily to save memory. + class LazyProxyIterator(f: => Iterator[Any]) extends Iterator[Any] { + lazy val proxy = f + override def hasNext: Boolean = proxy.hasNext + override def next(): Any = proxy.next() + } + new LazyProxyIterator(getIterator) + } else { + getIterator + } } def stop() { From b1f285359a12345d94ae59f979d9a75482e732ce Mon Sep 17 00:00:00 2001 From: tzolov Date: Tue, 3 Jun 2014 13:26:29 -0700 Subject: [PATCH 07/54] Add support for Pivotal HD in the Maven build: SPARK-1992 Allow Spark to build against particular Pivotal HD distributions. For example to build Spark against Pivotal HD 2.0.1 one can run: ``` mvn -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0-gphd-3.0.1.0 -DskipTests clean package ``` Author: tzolov Closes #942 from tzolov/master and squashes the following commits: bc3e05a [tzolov] Add support for Pivotal HD in the Maven build and SBT build: [SPARK-1992] --- pom.xml | 11 +++++++++++ project/SparkBuild.scala | 1 + 2 files changed, 12 insertions(+) diff --git a/pom.xml b/pom.xml index fe43a9518b52e..0a5ca9e72a296 100644 --- a/pom.xml +++ b/pom.xml @@ -206,6 +206,17 @@ false + + spring-releases + Spring Release Repository + http://repo.spring.io/libs-release + + true + + + false + + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 64c9441d8e3f8..c2a20d86b2e6b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -212,6 +212,7 @@ object SparkBuild extends Build { "JBoss Repository" at "https://repository.jboss.org/nexus/content/repositories/releases/", "MQTT Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/", "Cloudera Repository" at "http://repository.cloudera.com/artifactory/cloudera-repos/", + "Pivotal Repository" at "http://repo.spring.io/libs-release/", // For Sonatype publishing // "sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", // "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/", From 8edc9d0330c94b50e01956ae88693cff4e0977b2 Mon Sep 17 00:00:00 2001 From: Erik Selin Date: Tue, 3 Jun 2014 13:31:16 -0700 Subject: [PATCH 08/54] [SPARK-1468] Modify the partition function used by partitionBy. Make partitionBy use a tweaked version of hash as its default partition function since the python hash function does not consistently assign the same value to None across python processes. Associated JIRA at https://issues.apache.org/jira/browse/SPARK-1468 Author: Erik Selin Closes #371 from tyro89/consistent_hashing and squashes the following commits: 201c301 [Erik Selin] Make partitionBy use a tweaked version of hash as its default partition function since the python hash function does not consistently assign the same value to None across python processes. --- python/pyspark/rdd.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index f3b1f1a665e5a..1b3c460dd621e 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1062,7 +1062,7 @@ def rightOuterJoin(self, other, numPartitions=None): return python_right_outer_join(self, other, numPartitions) # TODO: add option to control map-side combining - def partitionBy(self, numPartitions, partitionFunc=hash): + def partitionBy(self, numPartitions, partitionFunc=None): """ Return a copy of the RDD partitioned using the specified partitioner. @@ -1073,6 +1073,9 @@ def partitionBy(self, numPartitions, partitionFunc=hash): """ if numPartitions is None: numPartitions = self.ctx.defaultParallelism + + if partitionFunc is None: + partitionFunc = lambda x: 0 if x is None else hash(x) # Transferring O(n) objects to Java is too expensive. Instead, we'll # form the hash buckets in Python, transferring O(numPartitions) objects # to Java. Each object is a (splitNumber, [objects]) pair. From aa41a522d821c989c65fa3f7f2a4d372e39bb958 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?baishuo=28=E7=99=BD=E7=A1=95=29?= Date: Tue, 3 Jun 2014 13:39:47 -0700 Subject: [PATCH 09/54] fix java.lang.ClassCastException MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit get Exception when run:bin/run-example org.apache.spark.examples.sql.RDDRelation Exception's detail is: Exception in thread "main" java.lang.ClassCastException: java.lang.Long cannot be cast to java.lang.Integer at scala.runtime.BoxesRunTime.unboxToInt(BoxesRunTime.java:106) at org.apache.spark.sql.catalyst.expressions.GenericRow.getInt(Row.scala:145) at org.apache.spark.examples.sql.RDDRelation$.main(RDDRelation.scala:49) at org.apache.spark.examples.sql.RDDRelation.main(RDDRelation.scala) change sql("SELECT COUNT(*) FROM records").collect().head.getInt(0) to sql("SELECT COUNT(*) FROM records").collect().head.getLong(0), then the Exception do not occur any more Author: baishuo(白硕) Closes #949 from baishuo/master and squashes the following commits: f4b319f [baishuo(白硕)] fix java.lang.ClassCastException --- .../main/scala/org/apache/spark/examples/sql/RDDRelation.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 61c460c6b1de8..63db688bfb8c0 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 @@ -43,7 +43,7 @@ object RDDRelation { sql("SELECT * FROM records").collect().foreach(println) // Aggregation queries are also supported. - val count = sql("SELECT COUNT(*) FROM records").collect().head.getInt(0) + val count = sql("SELECT COUNT(*) FROM records").collect().head.getLong(0) println(s"COUNT(*): $count") // The results of SQL queries are themselves RDDs and support all normal RDD functions. The From 894ecde04faa7e2054a40825a58b2e9cdaa93c70 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 3 Jun 2014 14:14:48 -0700 Subject: [PATCH 10/54] Synthetic GraphX Benchmark This PR accomplishes two things: 1. It introduces a Synthetic Benchmark application that generates an arbitrarily large log-normal graph and executes either PageRank or connected components on the graph. This can be used to profile GraphX system on arbitrary clusters without access to large graph datasets 2. This PR improves the implementation of the log-normal graph generator. Author: Joseph E. Gonzalez Author: Ankur Dave Closes #720 from jegonzal/graphx_synth_benchmark and squashes the following commits: e40812a [Ankur Dave] Exclude all of GraphX from compatibility checks vs. 1.0.0 bccccad [Ankur Dave] Fix long lines 374678a [Ankur Dave] Bugfix and style changes 1bdf39a [Joseph E. Gonzalez] updating options d943972 [Joseph E. Gonzalez] moving the benchmark application into the examples folder. f4f839a [Joseph E. Gonzalez] Creating a synthetic benchmark script. --- .../examples/graphx/SynthBenchmark.scala | 128 ++++++++++++++++++ .../spark/graphx/PartitionStrategy.scala | 9 ++ .../spark/graphx/util/GraphGenerators.scala | 41 ++++-- project/MimaExcludes.scala | 4 +- 4 files changed, 171 insertions(+), 11 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala new file mode 100644 index 0000000000000..551c339b19523 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala @@ -0,0 +1,128 @@ +/* + * 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.graphx + +import org.apache.spark.SparkContext._ +import org.apache.spark.graphx.PartitionStrategy +import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.graphx.util.GraphGenerators +import java.io.{PrintWriter, FileOutputStream} + +/** + * The SynthBenchmark application can be used to run various GraphX algorithms on + * synthetic log-normal graphs. The intent of this code is to enable users to + * profile the GraphX system without access to large graph datasets. + */ +object SynthBenchmark { + + /** + * To run this program use the following: + * + * MASTER=spark://foobar bin/run-example graphx.SynthBenchmark -app=pagerank + * + * Options: + * -app "pagerank" or "cc" for pagerank or connected components. (Default: pagerank) + * -niters the number of iterations of pagerank to use (Default: 10) + * -numVertices the number of vertices in the graph (Default: 1000000) + * -numEPart the number of edge partitions in the graph (Default: number of cores) + * -partStrategy the graph partitioning strategy to use + * -mu the mean parameter for the log-normal graph (Default: 4.0) + * -sigma the stdev parameter for the log-normal graph (Default: 1.3) + * -degFile the local file to save the degree information (Default: Empty) + */ + def main(args: Array[String]) { + val options = args.map { + arg => + arg.dropWhile(_ == '-').split('=') match { + case Array(opt, v) => (opt -> v) + case _ => throw new IllegalArgumentException("Invalid argument: " + arg) + } + } + + var app = "pagerank" + var niter = 10 + var numVertices = 100000 + var numEPart: Option[Int] = None + var partitionStrategy: Option[PartitionStrategy] = None + var mu: Double = 4.0 + var sigma: Double = 1.3 + var degFile: String = "" + + options.foreach { + case ("app", v) => app = v + case ("niter", v) => niter = v.toInt + case ("nverts", v) => numVertices = v.toInt + case ("numEPart", v) => numEPart = Some(v.toInt) + case ("partStrategy", v) => partitionStrategy = Some(PartitionStrategy.fromString(v)) + case ("mu", v) => mu = v.toDouble + case ("sigma", v) => sigma = v.toDouble + case ("degFile", v) => degFile = v + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + + val conf = new SparkConf() + .setAppName(s"GraphX Synth Benchmark (nverts = $numVertices, app = $app)") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + + val sc = new SparkContext(conf) + + // Create the graph + println(s"Creating graph...") + val unpartitionedGraph = GraphGenerators.logNormalGraph(sc, numVertices, + numEPart.getOrElse(sc.defaultParallelism), mu, sigma) + // Repartition the graph + val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)).cache() + + var startTime = System.currentTimeMillis() + val numEdges = graph.edges.count() + println(s"Done creating graph. Num Vertices = $numVertices, Num Edges = $numEdges") + val loadTime = System.currentTimeMillis() - startTime + + // Collect the degree distribution (if desired) + if (!degFile.isEmpty) { + val fos = new FileOutputStream(degFile) + val pos = new PrintWriter(fos) + val hist = graph.vertices.leftJoin(graph.degrees)((id, _, optDeg) => optDeg.getOrElse(0)) + .map(p => p._2).countByValue() + hist.foreach { + case (deg, count) => pos.println(s"$deg \t $count") + } + } + + // Run PageRank + startTime = System.currentTimeMillis() + if (app == "pagerank") { + println("Running PageRank") + val totalPR = graph.staticPageRank(niter).vertices.map(_._2).sum() + println(s"Total PageRank = $totalPR") + } else if (app == "cc") { + println("Running Connected Components") + val numComponents = graph.connectedComponents.vertices.map(_._2).distinct() + println(s"Number of components = $numComponents") + } + val runTime = System.currentTimeMillis() - startTime + + println(s"Num Vertices = $numVertices") + println(s"Num Edges = $numEdges") + println(s"Creation time = ${loadTime/1000.0} seconds") + println(s"Run time = ${runTime/1000.0} seconds") + + sc.stop() + } +} 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 1526ccef06fd4..ef412cfd4e6ea 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -119,4 +119,13 @@ object PartitionStrategy { math.abs((lower, higher).hashCode()) % numParts } } + + /** Returns the PartitionStrategy with the specified name. */ + def fromString(s: String): PartitionStrategy = s match { + case "RandomVertexCut" => RandomVertexCut + case "EdgePartition1D" => EdgePartition1D + case "EdgePartition2D" => EdgePartition2D + case "CanonicalRandomVertexCut" => CanonicalRandomVertexCut + case _ => throw new IllegalArgumentException("Invalid PartitionStrategy: " + s) + } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index a3c8de3f9068f..635514f09ece0 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -38,19 +38,42 @@ object GraphGenerators { val RMATa = 0.45 val RMATb = 0.15 val RMATd = 0.25 + /** * Generate a graph whose vertex out degree is log normal. + * + * The default values for mu and sigma are taken from the Pregel paper: + * + * Grzegorz Malewicz, Matthew H. Austern, Aart J.C Bik, James C. Dehnert, + * Ilan Horn, Naty Leiser, and Grzegorz Czajkowski. 2010. + * Pregel: a system for large-scale graph processing. SIGMOD '10. + * + * @param sc + * @param numVertices + * @param mu + * @param sigma + * @return */ - def logNormalGraph(sc: SparkContext, numVertices: Int): Graph[Int, Int] = { - // based on Pregel settings - val mu = 4 - val sigma = 1.3 - - val vertices: RDD[(VertexId, Int)] = sc.parallelize(0 until numVertices).map{ - src => (src, sampleLogNormal(mu, sigma, numVertices)) + def logNormalGraph(sc: SparkContext, numVertices: Int, numEParts: Int, + mu: Double = 4.0, sigma: Double = 1.3): Graph[Long, Int] = { + val vertices = sc.parallelize(0 until numVertices, numEParts).map { src => + // Initialize the random number generator with the source vertex id + val rand = new Random(src) + val degree = math.min(numVertices.toLong, math.exp(rand.nextGaussian() * sigma + mu).toLong) + (src.toLong, degree) } - val edges = vertices.flatMap { v => - generateRandomEdges(v._1.toInt, v._2, numVertices) + val edges = vertices.flatMap { case (src, degree) => + new Iterator[Edge[Int]] { + // Initialize the random number generator with the source vertex id + val rand = new Random(src) + var i = 0 + override def hasNext(): Boolean = { i < degree } + override def next(): Edge[Int] = { + val nextEdge = Edge[Int](src, rand.nextInt(numVertices), i) + i += 1 + nextEdge + } + } } Graph(vertices, edges, 0) } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index ecb389de5558f..fc9cbeaec6473 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -35,7 +35,8 @@ object MimaExcludes { val excludes = SparkBuild.SPARK_VERSION match { case v if v.startsWith("1.1") => - Seq() + Seq( + MimaBuild.excludeSparkPackage("graphx")) case v if v.startsWith("1.0") => Seq( MimaBuild.excludeSparkPackage("api.java"), @@ -58,4 +59,3 @@ object MimaExcludes { case _ => Seq() } } - From b1feb60209174433262de2a26d39616ba00edcc8 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Tue, 3 Jun 2014 14:54:26 -0700 Subject: [PATCH 11/54] [SPARK-1991] Support custom storage levels for vertices and edges This PR adds support for specifying custom storage levels for the vertices and edges of a graph. This enables GraphX to handle graphs larger than memory size by specifying MEMORY_AND_DISK and then repartitioning the graph to use many small partitions, each of which does fit in memory. Spark will then automatically load partitions from disk as needed. The user specifies the desired vertex and edge storage levels when building the graph by passing them to the graph constructor. These are then stored in the `targetStorageLevel` attribute of the VertexRDD and EdgeRDD respectively. Whenever GraphX needs to cache a VertexRDD or EdgeRDD (because it plans to use it more than once, for example), it uses the specified target storage level. Also, when the user calls `Graph#cache()`, the vertices and edges are persisted using their target storage levels. In order to facilitate propagating the target storage levels across VertexRDD and EdgeRDD operations, we remove raw calls to the constructors and instead introduce the `withPartitionsRDD` and `withTargetStorageLevel` methods. I tested this change by running PageRank and triangle count on a severely memory-constrained cluster (1 executor with 300 MB of memory, and a 1 GB graph). Before this PR, these algorithms used to fail with OutOfMemoryErrors. With this PR, and using the DISK_ONLY storage level, they succeed. Author: Ankur Dave Closes #946 from ankurdave/SPARK-1991 and squashes the following commits: ce17d95 [Ankur Dave] Move pickStorageLevel to StorageLevel.fromString ccaf06f [Ankur Dave] Shadow members in withXYZ() methods rather than using underscores c34abc0 [Ankur Dave] Exclude all of GraphX from compatibility checks vs. 1.0.0 c5ca068 [Ankur Dave] Revert "Exclude all of GraphX from binary compatibility checks" 34bcefb [Ankur Dave] Exclude all of GraphX from binary compatibility checks 6fdd137 [Ankur Dave] [SPARK-1991] Support custom storage levels for vertices and edges --- .../apache/spark/storage/StorageLevel.scala | 21 +++++ .../org/apache/spark/graphx/EdgeRDD.scala | 67 ++++++++++++++- .../scala/org/apache/spark/graphx/Graph.scala | 34 +++++--- .../org/apache/spark/graphx/GraphLoader.scala | 12 ++- .../org/apache/spark/graphx/VertexRDD.scala | 49 ++++++++--- .../apache/spark/graphx/impl/GraphImpl.scala | 55 +++++++------ .../graphx/impl/ReplicatedVertexView.scala | 6 +- .../apache/spark/graphx/lib/Analytics.scala | 82 +++++++++---------- 8 files changed, 229 insertions(+), 97 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 363de93e067b8..2d8ff1194a5dc 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -147,6 +147,27 @@ object StorageLevel { val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) val OFF_HEAP = new StorageLevel(false, false, true, false) + /** + * :: DeveloperApi :: + * Return the StorageLevel object with the specified name. + */ + @DeveloperApi + def fromString(s: String): StorageLevel = s match { + case "NONE" => NONE + case "DISK_ONLY" => DISK_ONLY + case "DISK_ONLY_2" => DISK_ONLY_2 + case "MEMORY_ONLY" => MEMORY_ONLY + case "MEMORY_ONLY_2" => MEMORY_ONLY_2 + case "MEMORY_ONLY_SER" => MEMORY_ONLY_SER + case "MEMORY_ONLY_SER_2" => MEMORY_ONLY_SER_2 + case "MEMORY_AND_DISK" => MEMORY_AND_DISK + case "MEMORY_AND_DISK_2" => MEMORY_AND_DISK_2 + case "MEMORY_AND_DISK_SER" => MEMORY_AND_DISK_SER + case "MEMORY_AND_DISK_SER_2" => MEMORY_AND_DISK_SER_2 + case "OFF_HEAP" => OFF_HEAP + case _ => throw new IllegalArgumentException("Invalid StorageLevel: " + s) + } + /** * :: DeveloperApi :: * Create a new StorageLevel object without setting useOffHeap. 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 a8fc095072512..899a3cbd62b60 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -24,6 +24,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.graphx.impl.EdgePartition +import org.apache.spark.graphx.impl.EdgePartitionBuilder /** * `EdgeRDD[ED, VD]` extends `RDD[Edge[ED]]` by storing the edges in columnar format on each @@ -32,7 +33,8 @@ import org.apache.spark.graphx.impl.EdgePartition * `impl.ReplicatedVertexView`. */ class EdgeRDD[@specialized ED: ClassTag, VD: ClassTag]( - val partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])]) + val partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])], + val targetStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) extends RDD[Edge[ED]](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { partitionsRDD.setName("EdgeRDD") @@ -58,6 +60,10 @@ class EdgeRDD[@specialized ED: ClassTag, VD: ClassTag]( override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect() + /** + * Persists the edge partitions at the specified storage level, ignoring any existing target + * storage level. + */ override def persist(newLevel: StorageLevel): this.type = { partitionsRDD.persist(newLevel) this @@ -68,9 +74,15 @@ class EdgeRDD[@specialized ED: ClassTag, VD: ClassTag]( this } + /** Persists the vertex partitions using `targetStorageLevel`, which defaults to MEMORY_ONLY. */ + override def cache(): this.type = { + partitionsRDD.persist(targetStorageLevel) + this + } + 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 => + this.withPartitionsRDD[ED2, VD2](partitionsRDD.mapPartitions({ iter => if (iter.hasNext) { val (pid, ep) = iter.next() Iterator(Tuple2(pid, f(pid, ep))) @@ -118,11 +130,60 @@ class EdgeRDD[@specialized ED: ClassTag, VD: ClassTag]( (f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3, VD] = { val ed2Tag = classTag[ED2] val ed3Tag = classTag[ED3] - new EdgeRDD[ED3, VD](partitionsRDD.zipPartitions(other.partitionsRDD, true) { + this.withPartitionsRDD[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))) }) } + + /** Replaces the vertex partitions while preserving all other properties of the VertexRDD. */ + private[graphx] def withPartitionsRDD[ED2: ClassTag, VD2: ClassTag]( + partitionsRDD: RDD[(PartitionID, EdgePartition[ED2, VD2])]): EdgeRDD[ED2, VD2] = { + new EdgeRDD(partitionsRDD, this.targetStorageLevel) + } + + /** + * Changes the target storage level while preserving all other properties of the + * EdgeRDD. Operations on the returned EdgeRDD will preserve this storage level. + * + * This does not actually trigger a cache; to do this, call + * [[org.apache.spark.graphx.EdgeRDD#cache]] on the returned EdgeRDD. + */ + private[graphx] def withTargetStorageLevel( + targetStorageLevel: StorageLevel): EdgeRDD[ED, VD] = { + new EdgeRDD(this.partitionsRDD, targetStorageLevel) + } + +} + +object EdgeRDD { + /** + * Creates an EdgeRDD from a set of edges. + * + * @tparam ED the edge attribute type + * @tparam VD the type of the vertex attributes that may be joined with the returned EdgeRDD + */ + def fromEdges[ED: ClassTag, VD: ClassTag](edges: RDD[Edge[ED]]): EdgeRDD[ED, VD] = { + val edgePartitions = edges.mapPartitionsWithIndex { (pid, iter) => + val builder = new EdgePartitionBuilder[ED, VD] + iter.foreach { e => + builder.add(e.srcId, e.dstId, e.attr) + } + Iterator((pid, builder.toEdgePartition)) + } + EdgeRDD.fromEdgePartitions(edgePartitions) + } + + /** + * Creates an EdgeRDD from already-constructed edge partitions. + * + * @tparam ED the edge attribute type + * @tparam VD the type of the vertex attributes that may be joined with the returned EdgeRDD + */ + def fromEdgePartitions[ED: ClassTag, VD: ClassTag]( + edgePartitions: RDD[(Int, EdgePartition[ED, VD])]): EdgeRDD[ED, VD] = { + new EdgeRDD(edgePartitions) + } } 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 dc5dac4fdad57..c4f9d6514cae3 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -80,7 +80,8 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab @transient val triplets: RDD[EdgeTriplet[VD, ED]] /** - * Caches the vertices and edges associated with this graph at the specified storage level. + * Caches the vertices and edges associated with this graph at the specified storage level, + * ignoring any target storage levels previously set. * * @param newLevel the level at which to cache the graph. * @@ -89,9 +90,9 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab def persist(newLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED] /** - * Caches the vertices and edges associated with this graph. This is used to - * pin a graph in memory enabling multiple queries to reuse the same - * construction process. + * Caches the vertices and edges associated with this graph at the previously-specified target + * storage levels, which default to `MEMORY_ONLY`. This is used to pin a graph in memory enabling + * multiple queries to reuse the same construction process. */ def cache(): Graph[VD, ED] @@ -358,9 +359,12 @@ object Graph { * Construct a graph from a collection of edges encoded as vertex id pairs. * * @param rawEdges a collection of edges in (src, dst) form + * @param defaultValue the vertex attributes with which to create vertices referenced by the edges * @param uniqueEdges if multiple identical edges are found they are combined and the edge * attribute is set to the sum. Otherwise duplicate edges are treated as separate. To enable * `uniqueEdges`, a [[PartitionStrategy]] must be provided. + * @param edgeStorageLevel the desired storage level at which to cache the edges if necessary + * @param vertexStorageLevel the desired storage level at which to cache the vertices if necessary * * @return a graph with edge attributes containing either the count of duplicate edges or 1 * (if `uniqueEdges` is `None`) and vertex attributes containing the total degree of each vertex. @@ -368,10 +372,12 @@ object Graph { def fromEdgeTuples[VD: ClassTag]( rawEdges: RDD[(VertexId, VertexId)], defaultValue: VD, - uniqueEdges: Option[PartitionStrategy] = None): Graph[VD, Int] = + uniqueEdges: Option[PartitionStrategy] = None, + edgeStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY, + vertexStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, Int] = { val edges = rawEdges.map(p => Edge(p._1, p._2, 1)) - val graph = GraphImpl(edges, defaultValue) + val graph = GraphImpl(edges, defaultValue, edgeStorageLevel, vertexStorageLevel) uniqueEdges match { case Some(p) => graph.partitionBy(p).groupEdges((a, b) => a + b) case None => graph @@ -383,14 +389,18 @@ object Graph { * * @param edges the RDD containing the set of edges in the graph * @param defaultValue the default vertex attribute to use for each vertex + * @param edgeStorageLevel the desired storage level at which to cache the edges if necessary + * @param vertexStorageLevel the desired storage level at which to cache the vertices if necessary * * @return a graph with edge attributes described by `edges` and vertices * given by all vertices in `edges` with value `defaultValue` */ def fromEdges[VD: ClassTag, ED: ClassTag]( edges: RDD[Edge[ED]], - defaultValue: VD): Graph[VD, ED] = { - GraphImpl(edges, defaultValue) + defaultValue: VD, + edgeStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY, + vertexStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED] = { + GraphImpl(edges, defaultValue, edgeStorageLevel, vertexStorageLevel) } /** @@ -405,12 +415,16 @@ object Graph { * @param edges the collection of edges in the graph * @param defaultVertexAttr the default vertex attribute to use for vertices that are * mentioned in edges but not in vertices + * @param edgeStorageLevel the desired storage level at which to cache the edges if necessary + * @param vertexStorageLevel the desired storage level at which to cache the vertices if necessary */ def apply[VD: ClassTag, ED: ClassTag]( vertices: RDD[(VertexId, VD)], edges: RDD[Edge[ED]], - defaultVertexAttr: VD = null.asInstanceOf[VD]): Graph[VD, ED] = { - GraphImpl(vertices, edges, defaultVertexAttr) + defaultVertexAttr: VD = null.asInstanceOf[VD], + edgeStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY, + vertexStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED] = { + GraphImpl(vertices, edges, defaultVertexAttr, edgeStorageLevel, vertexStorageLevel) } /** 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 389490c139848..2e814e34f9ad8 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -17,6 +17,7 @@ package org.apache.spark.graphx +import org.apache.spark.storage.StorageLevel import org.apache.spark.{Logging, SparkContext} import org.apache.spark.graphx.impl.{EdgePartitionBuilder, GraphImpl} @@ -48,12 +49,16 @@ object GraphLoader extends Logging { * @param canonicalOrientation whether to orient edges in the positive * direction * @param minEdgePartitions the number of partitions for the edge RDD + * @param edgeStorageLevel the desired storage level for the edge partitions. To set the vertex + * storage level, call [[org.apache.spark.graphx.Graph#persistVertices]]. */ def edgeListFile( sc: SparkContext, path: String, canonicalOrientation: Boolean = false, - minEdgePartitions: Int = 1) + minEdgePartitions: Int = 1, + edgeStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY, + vertexStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) : Graph[Int, Int] = { val startTime = System.currentTimeMillis @@ -78,12 +83,13 @@ object GraphLoader extends Logging { } } Iterator((pid, builder.toEdgePartition)) - }.cache().setName("GraphLoader.edgeListFile - edges (%s)".format(path)) + }.persist(edgeStorageLevel).setName("GraphLoader.edgeListFile - edges (%s)".format(path)) edges.count() logInfo("It took %d ms to load the edges".format(System.currentTimeMillis - startTime)) - GraphImpl.fromEdgePartitions(edges, defaultVertexAttr = 1) + GraphImpl.fromEdgePartitions(edges, defaultVertexAttr = 1, edgeStorageLevel = edgeStorageLevel, + vertexStorageLevel = vertexStorageLevel) } // end of edgeListFile } 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 8b910fbc5a423..f1b6df9a3025e 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -56,7 +56,8 @@ import org.apache.spark.graphx.impl.VertexRDDFunctions._ * @tparam VD the vertex attribute associated with each vertex in the set. */ class VertexRDD[@specialized VD: ClassTag]( - val partitionsRDD: RDD[ShippableVertexPartition[VD]]) + val partitionsRDD: RDD[ShippableVertexPartition[VD]], + val targetStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) extends RDD[(VertexId, VD)](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { require(partitionsRDD.partitioner.isDefined) @@ -66,7 +67,7 @@ class VertexRDD[@specialized VD: ClassTag]( * VertexRDD will be based on a different index and can no longer be quickly joined with this * RDD. */ - def reindex(): VertexRDD[VD] = new VertexRDD(partitionsRDD.map(_.reindex())) + def reindex(): VertexRDD[VD] = this.withPartitionsRDD(partitionsRDD.map(_.reindex())) override val partitioner = partitionsRDD.partitioner @@ -85,6 +86,10 @@ class VertexRDD[@specialized VD: ClassTag]( } setName("VertexRDD") + /** + * Persists the vertex partitions at the specified storage level, ignoring any existing target + * storage level. + */ override def persist(newLevel: StorageLevel): this.type = { partitionsRDD.persist(newLevel) this @@ -95,6 +100,12 @@ class VertexRDD[@specialized VD: ClassTag]( this } + /** Persists the vertex partitions at `targetStorageLevel`, which defaults to MEMORY_ONLY. */ + override def cache(): this.type = { + partitionsRDD.persist(targetStorageLevel) + this + } + /** The number of vertices in the RDD. */ override def count(): Long = { partitionsRDD.map(_.size).reduce(_ + _) @@ -114,7 +125,7 @@ class VertexRDD[@specialized VD: ClassTag]( f: ShippableVertexPartition[VD] => ShippableVertexPartition[VD2]) : VertexRDD[VD2] = { val newPartitionsRDD = partitionsRDD.mapPartitions(_.map(f), preservesPartitioning = true) - new VertexRDD(newPartitionsRDD) + this.withPartitionsRDD(newPartitionsRDD) } @@ -165,7 +176,7 @@ class VertexRDD[@specialized VD: ClassTag]( val otherPart = otherIter.next() Iterator(thisPart.diff(otherPart)) } - new VertexRDD(newPartitionsRDD) + this.withPartitionsRDD(newPartitionsRDD) } /** @@ -191,7 +202,7 @@ class VertexRDD[@specialized VD: ClassTag]( val otherPart = otherIter.next() Iterator(thisPart.leftJoin(otherPart)(f)) } - new VertexRDD(newPartitionsRDD) + this.withPartitionsRDD(newPartitionsRDD) } /** @@ -220,7 +231,7 @@ class VertexRDD[@specialized VD: ClassTag]( case other: VertexRDD[_] => leftZipJoin(other)(f) case _ => - new VertexRDD[VD3]( + this.withPartitionsRDD[VD3]( partitionsRDD.zipPartitions( other.copartitionWithVertices(this.partitioner.get), preservesPartitioning = true) { (partIter, msgs) => partIter.map(_.leftJoin(msgs)(f)) @@ -242,7 +253,7 @@ class VertexRDD[@specialized VD: ClassTag]( val otherPart = otherIter.next() Iterator(thisPart.innerJoin(otherPart)(f)) } - new VertexRDD(newPartitionsRDD) + this.withPartitionsRDD(newPartitionsRDD) } /** @@ -264,7 +275,7 @@ class VertexRDD[@specialized VD: ClassTag]( case other: VertexRDD[_] => innerZipJoin(other)(f) case _ => - new VertexRDD( + this.withPartitionsRDD( partitionsRDD.zipPartitions( other.copartitionWithVertices(this.partitioner.get), preservesPartitioning = true) { (partIter, msgs) => partIter.map(_.innerJoin(msgs)(f)) @@ -290,7 +301,7 @@ class VertexRDD[@specialized VD: ClassTag]( val parts = partitionsRDD.zipPartitions(shuffled, true) { (thisIter, msgIter) => thisIter.map(_.aggregateUsingIndex(msgIter, reduceFunc)) } - new VertexRDD[VD2](parts) + this.withPartitionsRDD[VD2](parts) } /** @@ -309,7 +320,25 @@ class VertexRDD[@specialized VD: ClassTag]( if (routingTableIter.hasNext) routingTableIter.next() else RoutingTablePartition.empty partIter.map(_.withRoutingTable(routingTable)) } - new VertexRDD(vertexPartitions) + this.withPartitionsRDD(vertexPartitions) + } + + /** Replaces the vertex partitions while preserving all other properties of the VertexRDD. */ + private[graphx] def withPartitionsRDD[VD2: ClassTag]( + partitionsRDD: RDD[ShippableVertexPartition[VD2]]): VertexRDD[VD2] = { + new VertexRDD(partitionsRDD, this.targetStorageLevel) + } + + /** + * Changes the target storage level while preserving all other properties of the + * VertexRDD. Operations on the returned VertexRDD will preserve this storage level. + * + * This does not actually trigger a cache; to do this, call + * [[org.apache.spark.graphx.VertexRDD#cache]] on the returned VertexRDD. + */ + private[graphx] def withTargetStorageLevel( + targetStorageLevel: StorageLevel): VertexRDD[VD] = { + new VertexRDD(this.partitionsRDD, targetStorageLevel) } /** Generates an RDD of vertex attributes suitable for shipping to the edge partitions. */ 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 1649b244d2881..59d9a8808e56e 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 @@ -61,7 +61,11 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( this } - override def cache(): Graph[VD, ED] = persist(StorageLevel.MEMORY_ONLY) + override def cache(): Graph[VD, ED] = { + vertices.cache() + replicatedVertexView.edges.cache() + this + } override def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] = { vertices.unpersist(blocking) @@ -70,10 +74,10 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { - val numPartitions = replicatedVertexView.edges.partitions.size + val numPartitions = edges.partitions.size val edTag = classTag[ED] val vdTag = classTag[VD] - val newEdges = new EdgeRDD(replicatedVertexView.edges.map { e => + val newEdges = edges.withPartitionsRDD(edges.map { e => val part: PartitionID = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) // Should we be using 3-tuple or an optimized class @@ -256,24 +260,33 @@ 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] = { - fromEdgeRDD(createEdgeRDD(edges), defaultVertexAttr) + defaultVertexAttr: VD, + edgeStorageLevel: StorageLevel, + vertexStorageLevel: StorageLevel): GraphImpl[VD, ED] = { + fromEdgeRDD(EdgeRDD.fromEdges(edges), defaultVertexAttr, edgeStorageLevel, vertexStorageLevel) } /** Create a graph from EdgePartitions, setting referenced vertices to `defaultVertexAttr`. */ def fromEdgePartitions[VD: ClassTag, ED: ClassTag]( edgePartitions: RDD[(PartitionID, EdgePartition[ED, VD])], - defaultVertexAttr: VD): GraphImpl[VD, ED] = { - fromEdgeRDD(new EdgeRDD(edgePartitions), defaultVertexAttr) + defaultVertexAttr: VD, + edgeStorageLevel: StorageLevel, + vertexStorageLevel: StorageLevel): GraphImpl[VD, ED] = { + fromEdgeRDD(EdgeRDD.fromEdgePartitions(edgePartitions), defaultVertexAttr, edgeStorageLevel, + vertexStorageLevel) } /** 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)(classTag[ED], classTag[VD]).cache() + defaultVertexAttr: VD, + edgeStorageLevel: StorageLevel, + vertexStorageLevel: StorageLevel): GraphImpl[VD, ED] = { + val edgeRDD = EdgeRDD.fromEdges(edges)(classTag[ED], classTag[VD]) + .withTargetStorageLevel(edgeStorageLevel).cache() val vertexRDD = VertexRDD(vertices, edgeRDD, defaultVertexAttr) + .withTargetStorageLevel(vertexStorageLevel).cache() GraphImpl(vertexRDD, edgeRDD) } @@ -309,23 +322,13 @@ object GraphImpl { */ 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, VD] - iter.foreach { e => - builder.add(e.srcId, e.dstId, e.attr) - } - Iterator((pid, builder.toEdgePartition)) - } - new EdgeRDD(edgePartitions) + defaultVertexAttr: VD, + edgeStorageLevel: StorageLevel, + vertexStorageLevel: StorageLevel): GraphImpl[VD, ED] = { + val edgesCached = edges.withTargetStorageLevel(edgeStorageLevel).cache() + val vertices = VertexRDD.fromEdges(edgesCached, edgesCached.partitions.size, defaultVertexAttr) + .withTargetStorageLevel(vertexStorageLevel) + fromExistingRDDs(vertices, edgesCached) } } // end of object GraphImpl 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 3a0bba1b93b41..86b366eb9202b 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 @@ -69,7 +69,7 @@ class ReplicatedVertexView[VD: ClassTag, ED: ClassTag]( .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) { + val newEdges = edges.withPartitionsRDD(edges.partitionsRDD.zipPartitions(shippedVerts) { (ePartIter, shippedVertsIter) => ePartIter.map { case (pid, edgePartition) => (pid, edgePartition.updateVertices(shippedVertsIter.flatMap(_._2.iterator))) @@ -91,7 +91,7 @@ class ReplicatedVertexView[VD: ClassTag, ED: ClassTag]( .setName("ReplicatedVertexView.withActiveSet - shippedActives (broadcast)") .partitionBy(edges.partitioner.get) - val newEdges = new EdgeRDD(edges.partitionsRDD.zipPartitions(shippedActives) { + val newEdges = edges.withPartitionsRDD(edges.partitionsRDD.zipPartitions(shippedActives) { (ePartIter, shippedActivesIter) => ePartIter.map { case (pid, edgePartition) => (pid, edgePartition.withActiveSet(shippedActivesIter.flatMap(_._2.iterator))) @@ -111,7 +111,7 @@ class ReplicatedVertexView[VD: ClassTag, ED: ClassTag]( hasSrcId, hasDstId)) .partitionBy(edges.partitioner.get) - val newEdges = new EdgeRDD(edges.partitionsRDD.zipPartitions(shippedVerts) { + val newEdges = edges.withPartitionsRDD(edges.partitionsRDD.zipPartitions(shippedVerts) { (ePartIter, shippedVertsIter) => ePartIter.map { case (pid, edgePartition) => (pid, edgePartition.updateVertices(shippedVertsIter.flatMap(_._2.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 069e042ed94a3..c1513a00453cf 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 @@ -17,7 +17,9 @@ package org.apache.spark.graphx.lib +import scala.collection.mutable import org.apache.spark._ +import org.apache.spark.storage.StorageLevel import org.apache.spark.graphx._ import org.apache.spark.graphx.PartitionStrategy._ @@ -28,18 +30,20 @@ object Analytics extends Logging { def main(args: Array[String]): Unit = { if (args.length < 2) { - System.err.println("Usage: Analytics [other options]") + System.err.println( + "Usage: Analytics --numEPart= [other options]") System.exit(1) } val taskType = args(0) val fname = args(1) - val options = args.drop(2).map { arg => + val optionsList = args.drop(2).map { arg => arg.dropWhile(_ == '-').split('=') match { case Array(opt, v) => (opt -> v) case _ => throw new IllegalArgumentException("Invalid argument: " + arg) } } + val options = mutable.Map(optionsList: _*) def pickPartitioner(v: String): PartitionStrategy = { // TODO: Use reflection rather than listing all the partitioning strategies here. @@ -57,20 +61,24 @@ object Analytics extends Logging { .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") .set("spark.locality.wait", "100000") + val numEPart = options.remove("numEPart").map(_.toInt).getOrElse { + println("Set the number of edge partitions using --numEPart.") + sys.exit(1) + } + val partitionStrategy: Option[PartitionStrategy] = options.remove("partStrategy") + .map(pickPartitioner(_)) + val edgeStorageLevel = options.remove("edgeStorageLevel") + .map(StorageLevel.fromString(_)).getOrElse(StorageLevel.MEMORY_ONLY) + val vertexStorageLevel = options.remove("vertexStorageLevel") + .map(StorageLevel.fromString(_)).getOrElse(StorageLevel.MEMORY_ONLY) + taskType match { case "pagerank" => - var tol: Float = 0.001F - 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) + val tol = options.remove("tol").map(_.toFloat).getOrElse(0.001F) + val outFname = options.remove("output").getOrElse("") + val numIterOpt = options.remove("numIter").map(_.toInt) + + options.foreach { case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } @@ -81,7 +89,9 @@ object Analytics extends Logging { val sc = new SparkContext(conf.setAppName("PageRank(" + fname + ")")) val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart).cache() + minEdgePartitions = numEPart, + edgeStorageLevel = edgeStorageLevel, + vertexStorageLevel = vertexStorageLevel).cache() val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) println("GRAPHX: Number of vertices " + graph.vertices.count) @@ -102,32 +112,19 @@ object Analytics extends Logging { sc.stop() case "cc" => - var numIter = Int.MaxValue - var numVPart = 4 - var numEPart = 4 - var isDynamic = false - var partitionStrategy: Option[PartitionStrategy] = None - - options.foreach{ - case ("numIter", v) => numIter = v.toInt - case ("dynamic", v) => isDynamic = v.toBoolean - case ("numEPart", v) => numEPart = v.toInt - case ("numVPart", v) => numVPart = v.toInt - case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) + options.foreach { case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } - if (!isDynamic && numIter == Int.MaxValue) { - println("Set number of iterations!") - sys.exit(1) - } println("======================================") println("| Connected Components |") println("======================================") val sc = new SparkContext(conf.setAppName("ConnectedComponents(" + fname + ")")) val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart).cache() + minEdgePartitions = numEPart, + edgeStorageLevel = edgeStorageLevel, + vertexStorageLevel = vertexStorageLevel).cache() val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) val cc = ConnectedComponents.run(graph) @@ -135,24 +132,25 @@ object Analytics extends Logging { sc.stop() case "triangles" => - var numEPart = 4 - // TriangleCount requires the graph to be partitioned - var partitionStrategy: PartitionStrategy = RandomVertexCut - - options.foreach{ - case ("numEPart", v) => numEPart = v.toInt - case ("partStrategy", v) => partitionStrategy = pickPartitioner(v) + options.foreach { case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } + println("======================================") println("| Triangle Count |") println("======================================") + val sc = new SparkContext(conf.setAppName("TriangleCount(" + fname + ")")) - val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true, - minEdgePartitions = numEPart).partitionBy(partitionStrategy).cache() + val graph = GraphLoader.edgeListFile(sc, fname, + canonicalOrientation = true, + minEdgePartitions = numEPart, + edgeStorageLevel = edgeStorageLevel, + vertexStorageLevel = vertexStorageLevel) + // TriangleCount requires the graph to be partitioned + .partitionBy(partitionStrategy.getOrElse(RandomVertexCut)).cache() val triangles = TriangleCount.run(graph) println("Triangles: " + triangles.vertices.map { - case (vid,data) => data.toLong + case (vid, data) => data.toLong }.reduce(_ + _) / 3) sc.stop() From f4dd665c85713d4c09731080fca58aee0fa2a85a Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 3 Jun 2014 18:10:58 -0700 Subject: [PATCH 12/54] Fixed a typo in RowMatrix.scala Author: DB Tsai Closes #959 from dbtsai/dbtsai-typo and squashes the following commits: fab0e0e [DB Tsai] Fixed typo --- .../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 07dfadf2f7869..00d0b18c27a8d 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 @@ -416,7 +416,7 @@ class RowMatrix( mat } - /** Updates or verfires the number of rows. */ + /** Updates or verifies the number of rows. */ private def updateNumRows(m: Long) { if (nRows <= 0) { nRows == m From 21e40ed88bf2c205c3d7f947fde5d5a6f3e29f7f Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Tue, 3 Jun 2014 18:18:25 -0700 Subject: [PATCH 13/54] [SPARK-1161] Add saveAsPickleFile and SparkContext.pickleFile in Python Author: Kan Zhang Closes #755 from kanzhang/SPARK-1161 and squashes the following commits: 24ed8a2 [Kan Zhang] [SPARK-1161] Fixing doc tests 44e0615 [Kan Zhang] [SPARK-1161] Adding an optional batchSize with default value 10 d929429 [Kan Zhang] [SPARK-1161] Add saveAsObjectFile and SparkContext.objectFile in Python --- python/pyspark/context.py | 14 ++++++++++++++ python/pyspark/rdd.py | 33 +++++++++++++++++++++++++-------- 2 files changed, 39 insertions(+), 8 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 9ae9305d4f02e..211918f5a05ec 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -271,6 +271,20 @@ def parallelize(self, c, numSlices=None): jrdd = readRDDFromFile(self._jsc, tempFile.name, numSlices) return RDD(jrdd, self, serializer) + def pickleFile(self, name, minPartitions=None): + """ + Load an RDD previously saved using L{RDD.saveAsPickleFile} method. + + >>> tmpFile = NamedTemporaryFile(delete=True) + >>> tmpFile.close() + >>> sc.parallelize(range(10)).saveAsPickleFile(tmpFile.name, 5) + >>> sorted(sc.pickleFile(tmpFile.name, 3).collect()) + [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] + """ + minPartitions = minPartitions or self.defaultMinPartitions + return RDD(self._jsc.objectFile(name, minPartitions), self, + BatchedSerializer(PickleSerializer())) + def textFile(self, name, minPartitions=None): """ Read a text file from HDFS, a local file system (available on all diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 1b3c460dd621e..ca0a95578fd28 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -33,7 +33,8 @@ from random import Random from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ - BatchedSerializer, CloudPickleSerializer, PairDeserializer, pack_long + BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ + PickleSerializer, pack_long from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup from pyspark.statcounter import StatCounter @@ -427,11 +428,14 @@ def intersection(self, other): .filter(lambda x: (len(x[1][0]) != 0) and (len(x[1][1]) != 0)) \ .keys() - def _reserialize(self): - if self._jrdd_deserializer == self.ctx.serializer: + def _reserialize(self, serializer=None): + serializer = serializer or self.ctx.serializer + if self._jrdd_deserializer == serializer: return self else: - return self.map(lambda x: x, preservesPartitioning=True) + converted = self.map(lambda x: x, preservesPartitioning=True) + converted._jrdd_deserializer = serializer + return converted def __add__(self, other): """ @@ -897,6 +901,20 @@ def first(self): """ return self.take(1)[0] + def saveAsPickleFile(self, path, batchSize=10): + """ + Save this RDD as a SequenceFile of serialized objects. The serializer used is + L{pyspark.serializers.PickleSerializer}, default batch size is 10. + + >>> tmpFile = NamedTemporaryFile(delete=True) + >>> tmpFile.close() + >>> sc.parallelize([1, 2, 'spark', 'rdd']).saveAsPickleFile(tmpFile.name, 3) + >>> sorted(sc.pickleFile(tmpFile.name, 5).collect()) + [1, 2, 'rdd', 'spark'] + """ + self._reserialize(BatchedSerializer(PickleSerializer(), + batchSize))._jrdd.saveAsObjectFile(path) + def saveAsTextFile(self, path): """ Save this RDD as a text file, using string representations of elements. @@ -1421,10 +1439,9 @@ def _jrdd(self): if self._jrdd_val: return self._jrdd_val if self._bypass_serializer: - serializer = NoOpSerializer() - else: - serializer = self.ctx.serializer - command = (self.func, self._prev_jrdd_deserializer, serializer) + self._jrdd_deserializer = NoOpSerializer() + command = (self.func, self._prev_jrdd_deserializer, + self._jrdd_deserializer) pickled_command = CloudPickleSerializer().dumps(command) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], From 1faef149f763f4a54aaa6e17043d0a628ae338a0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 3 Jun 2014 18:37:40 -0700 Subject: [PATCH 14/54] SPARK-1941: Update streamlib to 2.7.0 and use HyperLogLogPlus instead of HyperLogLog. I also corrected some errors made in the previous HLL count approximate API, including relativeSD wasn't really a measure for error (and we used it to test error bounds in test results). Author: Reynold Xin Closes #897 from rxin/hll and squashes the following commits: 4d83f41 [Reynold Xin] New error bound and non-randomness. f154ea0 [Reynold Xin] Added a comment on the value bound for testing. e367527 [Reynold Xin] One more round of code review. 41e649a [Reynold Xin] Update final mima list. 9e320c8 [Reynold Xin] Incorporate code review feedback. e110d70 [Reynold Xin] Merge branch 'master' into hll 354deb8 [Reynold Xin] Added comment on the Mima exclude rules. acaa524 [Reynold Xin] Added the right exclude rules in MimaExcludes. 6555bfe [Reynold Xin] Added a default method and re-arranged MimaExcludes. 1db1522 [Reynold Xin] Excluded util.SerializableHyperLogLog from MIMA check. 9221b27 [Reynold Xin] Merge branch 'master' into hll 88cfe77 [Reynold Xin] Updated documentation and restored the old incorrect API to maintain API compatibility. 1294be6 [Reynold Xin] Updated HLL+. e7786cb [Reynold Xin] Merge branch 'master' into hll c0ef0c2 [Reynold Xin] SPARK-1941: Update streamlib to 2.7.0 and use HyperLogLogPlus instead of HyperLogLog. --- .../apache/spark/api/java/JavaPairRDD.scala | 51 ++++++----- .../apache/spark/api/java/JavaRDDLike.scala | 12 +-- .../apache/spark/rdd/PairRDDFunctions.scala | 90 ++++++++++++++----- .../main/scala/org/apache/spark/rdd/RDD.scala | 53 ++++++++--- .../spark/util/SerializableHyperLogLog.scala | 52 ----------- .../java/org/apache/spark/JavaAPISuite.java | 10 +-- .../spark/rdd/PairRDDFunctionsSuite.scala | 22 ++--- .../scala/org/apache/spark/rdd/RDDSuite.scala | 6 +- pom.xml | 4 +- project/MimaExcludes.scala | 22 ++++- project/SparkBuild.scala | 2 +- 11 files changed, 189 insertions(+), 135 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala 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 4c8f9ed6fbc02..7dcfbf741c4f1 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 @@ -672,38 +672,47 @@ class JavaPairRDD[K, V](val rdd: 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 - * Partitioner to partition the output RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. + * @param partitioner partitioner of the resulting RDD. */ - def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): JavaRDD[(K, Long)] = { - rdd.countApproxDistinctByKey(relativeSD, partitioner) + def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): JavaPairRDD[K, Long] = + { + fromRDD(rdd.countApproxDistinctByKey(relativeSD, partitioner)) } /** - * 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 - * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism - * level. + * Return approximate number of distinct values for each key in this RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. + * @param numPartitions number of partitions of the resulting RDD. */ - def countApproxDistinctByKey(relativeSD: Double = 0.05): JavaRDD[(K, Long)] = { - rdd.countApproxDistinctByKey(relativeSD) + def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): JavaPairRDD[K, Long] = { + fromRDD(rdd.countApproxDistinctByKey(relativeSD, numPartitions)) } - /** * 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 - * output RDD into numPartitions. * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. */ - def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): JavaRDD[(K, Long)] = { - rdd.countApproxDistinctByKey(relativeSD, numPartitions) + def countApproxDistinctByKey(relativeSD: Double): JavaPairRDD[K, Long] = { + fromRDD(rdd.countApproxDistinctByKey(relativeSD)) } /** Assign a name to this RDD */ 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 619bfd75be8eb..330569a8d8837 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 @@ -560,12 +560,14 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return approximate number of distinct elements in the 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 - * relativeSD is 0.05. + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. */ - def countApproxDistinct(relativeSD: Double = 0.05): Long = rdd.countApproxDistinct(relativeSD) + def countApproxDistinct(relativeSD: Double): Long = rdd.countApproxDistinct(relativeSD) def name(): String = rdd.name 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 223fef79261d0..f2ce3cbd47f93 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -28,7 +28,7 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag -import com.clearspring.analytics.stream.cardinality.HyperLogLog +import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.io.SequenceFile.CompressionType @@ -46,7 +46,6 @@ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer -import org.apache.spark.util.SerializableHyperLogLog /** * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. @@ -214,39 +213,88 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) } /** + * :: Experimental :: + * * 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 vice versa. Uses the provided - * Partitioner to partition the output RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero `sp > p` + * would trigger sparse representation of registers, which may reduce the memory consumption + * and increase accuracy when the cardinality is small. + * + * @param p The precision value for the normal set. + * `p` must be a value between 4 and `sp` if `sp` is not zero (32 max). + * @param sp The precision value for the sparse set, between 0 and 32. + * If `sp` equals 0, the sparse representation is skipped. + * @param partitioner Partitioner to use for the resulting RDD. */ - def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)] = { - val createHLL = (v: V) => new SerializableHyperLogLog(new HyperLogLog(relativeSD)).add(v) - val mergeValueHLL = (hll: SerializableHyperLogLog, v: V) => hll.add(v) - val mergeHLL = (h1: SerializableHyperLogLog, h2: SerializableHyperLogLog) => h1.merge(h2) + @Experimental + def countApproxDistinctByKey(p: Int, sp: Int, partitioner: Partitioner): RDD[(K, Long)] = { + require(p >= 4, s"p ($p) must be >= 4") + require(sp <= 32, s"sp ($sp) must be <= 32") + require(sp == 0 || p <= sp, s"p ($p) cannot be greater than sp ($sp)") + val createHLL = (v: V) => { + val hll = new HyperLogLogPlus(p, sp) + hll.offer(v) + hll + } + val mergeValueHLL = (hll: HyperLogLogPlus, v: V) => { + hll.offer(v) + hll + } + val mergeHLL = (h1: HyperLogLogPlus, h2: HyperLogLogPlus) => { + h1.addAll(h2) + h1 + } + + combineByKey(createHLL, mergeValueHLL, mergeHLL, partitioner).mapValues(_.cardinality()) + } - combineByKey(createHLL, mergeValueHLL, mergeHLL, partitioner).mapValues(_.value.cardinality()) + /** + * Return approximate number of distinct values for each key in this RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. + * @param partitioner partitioner of the resulting RDD + */ + def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)] = { + require(relativeSD > 0.000017, s"accuracy ($relativeSD) must be greater than 0.000017") + val p = math.ceil(2.0 * math.log(1.054 / relativeSD) / math.log(2)).toInt + assert(p <= 32) + countApproxDistinctByKey(if (p < 4) 4 else p, 0, partitioner) } /** * 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 vice versa. HashPartitions the - * output RDD into numPartitions. * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. + * @param numPartitions number of partitions of the resulting RDD */ def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): RDD[(K, Long)] = { countApproxDistinctByKey(relativeSD, new HashPartitioner(numPartitions)) } /** - * 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 vice versa. The default value of - * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism - * level. + * Return approximate number of distinct values for each key in this RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. */ def countApproxDistinctByKey(relativeSD: Double = 0.05): RDD[(K, Long)] = { countApproxDistinctByKey(relativeSD, defaultPartitioner(self)) 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 aa03e9276fb34..585b2f76afa65 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -19,12 +19,11 @@ package org.apache.spark.rdd import java.util.Random -import scala.collection.Map -import scala.collection.mutable +import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer import scala.reflect.{classTag, ClassTag} -import com.clearspring.analytics.stream.cardinality.HyperLogLog +import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus import org.apache.hadoop.io.BytesWritable import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.io.NullWritable @@ -41,7 +40,7 @@ import org.apache.spark.partial.CountEvaluator 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.{BoundedPriorityQueue, Utils} import org.apache.spark.util.collection.OpenHashMap import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler} @@ -921,15 +920,49 @@ 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 - * (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 - * relativeSD is 0.05. + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero `sp > p` + * would trigger sparse representation of registers, which may reduce the memory consumption + * and increase accuracy when the cardinality is small. + * + * @param p The precision value for the normal set. + * `p` must be a value between 4 and `sp` if `sp` is not zero (32 max). + * @param sp The precision value for the sparse set, between 0 and 32. + * If `sp` equals 0, the sparse representation is skipped. */ @Experimental + def countApproxDistinct(p: Int, sp: Int): Long = { + require(p >= 4, s"p ($p) must be greater than 0") + require(sp <= 32, s"sp ($sp) cannot be greater than 32") + require(sp == 0 || p <= sp, s"p ($p) cannot be greater than sp ($sp)") + val zeroCounter = new HyperLogLogPlus(p, sp) + aggregate(zeroCounter)( + (hll: HyperLogLogPlus, v: T) => { + hll.offer(v) + hll + }, + (h1: HyperLogLogPlus, h2: HyperLogLogPlus) => { + h1.addAll(h2) + h2 + }).cardinality() + } + + /** + * Return approximate number of distinct elements in the RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. + */ def countApproxDistinct(relativeSD: Double = 0.05): Long = { - val zeroCounter = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) - aggregate(zeroCounter)(_.add(_), _.merge(_)).value.cardinality() + val p = math.ceil(2.0 * math.log(1.054 / relativeSD) / math.log(2)).toInt + countApproxDistinct(p, 0) } /** diff --git a/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala b/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala deleted file mode 100644 index 21a88eea3bbc2..0000000000000 --- a/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala +++ /dev/null @@ -1,52 +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.util - -import java.io.{Externalizable, ObjectInput, ObjectOutput} - -import com.clearspring.analytics.stream.cardinality.{HyperLogLog, ICardinality} - -/** - * A wrapper around [[com.clearspring.analytics.stream.cardinality.HyperLogLog]] that is - * serializable. - */ -private[spark] -class SerializableHyperLogLog(var value: ICardinality) extends Externalizable { - - def this() = this(null) // For deserialization - - def merge(other: SerializableHyperLogLog) = new SerializableHyperLogLog(value.merge(other.value)) - - def add[T](elem: T) = { - this.value.offer(elem) - this - } - - def readExternal(in: ObjectInput) { - val byteLength = in.readInt() - val bytes = new Array[Byte](byteLength) - in.readFully(bytes) - value = HyperLogLog.Builder.build(bytes) - } - - def writeExternal(out: ObjectOutput) { - val bytes = value.getBytes() - out.writeInt(bytes.length) - out.write(bytes) - } -} diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 7193223addf66..b78309f81cb8c 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -1028,27 +1028,23 @@ public void countApproxDistinct() { arrayData.add(i % size); } JavaRDD simpleRdd = sc.parallelize(arrayData, 10); - Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.2) - size) / (size * 1.0)) < 0.2); - Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.05) - size) / (size * 1.0)) <= 0.05); - Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.01) - size) / (size * 1.0)) <= 0.01); + Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.05) - size) / (size * 1.0)) <= 0.1); } @Test public void countApproxDistinctByKey() { - double relativeSD = 0.001; - List> arrayData = new ArrayList>(); for (int i = 10; i < 100; i++) for (int j = 0; j < i; j++) arrayData.add(new Tuple2(i, j)); JavaPairRDD pairRdd = sc.parallelizePairs(arrayData); - List> res = pairRdd.countApproxDistinctByKey(relativeSD).collect(); + List> res = pairRdd.countApproxDistinctByKey(8, 0).collect(); for (Tuple2 resItem : res) { double count = (double)resItem._1(); Long resCount = (Long)resItem._2(); Double error = Math.abs((resCount - count) / count); - Assert.assertTrue(error < relativeSD); + Assert.assertTrue(error < 0.1); } } 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 1230565ea5b7e..9ddafc451878d 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -119,28 +119,30 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { * relatively tight error bounds to check correctness of functionality rather than checking * whether the approximation conforms with the requested bound. */ - val relativeSD = 0.001 + val p = 20 + val sp = 0 + // When p = 20, the relative accuracy is about 0.001. So with high probability, the + // relative error should be smaller than the threshold 0.01 we use here. + val relativeSD = 0.01 // For each value i, there are i tuples with first element equal to i. // Therefore, the expected count for key i would be i. val stacked = (1 to 100).flatMap(i => (1 to i).map(j => (i, j))) val rdd1 = sc.parallelize(stacked) - val counted1 = rdd1.countApproxDistinctByKey(relativeSD).collect() - counted1.foreach{ - case(k, count) => assert(error(count, k) < relativeSD) - } + val counted1 = rdd1.countApproxDistinctByKey(p, sp).collect() + counted1.foreach { case (k, count) => assert(error(count, k) < relativeSD) } - val rnd = new Random() + val rnd = new Random(42) // The expected count for key num would be num val randStacked = (1 to 100).flatMap { i => - val num = rnd.nextInt % 500 + val num = rnd.nextInt() % 500 (1 to num).map(j => (num, j)) } val rdd2 = sc.parallelize(randStacked) - val counted2 = rdd2.countApproxDistinctByKey(relativeSD, 4).collect() - counted2.foreach{ - case(k, count) => assert(error(count, k) < relativeSD) + val counted2 = rdd2.countApproxDistinctByKey(relativeSD).collect() + counted2.foreach { case (k, count) => + assert(error(count, k) < relativeSD, s"${error(count, k)} < $relativeSD") } } 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 e686068f7a99a..bbd0c14178368 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -73,10 +73,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { val size = 100 val uniformDistro = for (i <- 1 to 100000) yield i % size val simpleRdd = sc.makeRDD(uniformDistro) - assert(error(simpleRdd.countApproxDistinct(0.2), size) < 0.2) - assert(error(simpleRdd.countApproxDistinct(0.05), size) < 0.05) - assert(error(simpleRdd.countApproxDistinct(0.01), size) < 0.01) - assert(error(simpleRdd.countApproxDistinct(0.001), size) < 0.001) + assert(error(simpleRdd.countApproxDistinct(4, 0), size) < 0.4) + assert(error(simpleRdd.countApproxDistinct(8, 0), size) < 0.1) } test("SparkContext.union") { diff --git a/pom.xml b/pom.xml index 0a5ca9e72a296..fcd6f66b4414a 100644 --- a/pom.xml +++ b/pom.xml @@ -300,9 +300,9 @@ com.clearspring.analytics stream - 2.5.1 + 2.7.0 - + it.unimi.dsi fastutil diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index fc9cbeaec6473..fadf6a4d8bb4a 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -16,7 +16,6 @@ */ import com.typesafe.tools.mima.core._ -import com.typesafe.tools.mima.core.ProblemFilters._ /** * Additional excludes for checking of Spark's binary compatibility. @@ -35,8 +34,27 @@ object MimaExcludes { val excludes = SparkBuild.SPARK_VERSION match { case v if v.startsWith("1.1") => + Seq(MimaBuild.excludeSparkPackage("graphx")) ++ Seq( - MimaBuild.excludeSparkPackage("graphx")) + // We made a mistake earlier (ed06500d3) in the Java API to use default parameter values + // for countApproxDistinct* functions, which does not work in Java. We later removed + // them, and use the following to tell Mima to not care about them. + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.api.java.JavaPairRDD.countApproxDistinctByKey"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.api.java.JavaPairRDD.countApproxDistinctByKey"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaPairRDD.countApproxDistinct$default$1"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaPairRDD.countApproxDistinctByKey$default$1"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDD.countApproxDistinct$default$1"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDDLike.countApproxDistinct$default$1"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaDoubleRDD.countApproxDistinct$default$1") + ) ++ + MimaBuild.excludeSparkClass("util.SerializableHyperLogLog") case v if v.startsWith("1.0") => Seq( MimaBuild.excludeSparkPackage("api.java"), diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c2a20d86b2e6b..efb0b9319be13 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -362,7 +362,7 @@ object SparkBuild extends Build { "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" excludeAll(excludeFastutil), + "com.clearspring.analytics" % "stream" % "2.7.0" excludeAll(excludeFastutil), // Only HyperLogLogPlus is used, which does not depend on fastutil. "org.spark-project" % "pyrolite" % "2.0.1", "net.sf.py4j" % "py4j" % "0.8.1" ), From e8d93ee5284cb6a1d4551effe91ee8d233323329 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 3 Jun 2014 18:53:13 -0700 Subject: [PATCH 15/54] use env default python in merge_spark_pr.py A minor change to use env default python instead of fixed `/usr/bin/python`. Author: Xiangrui Meng Closes #965 from mengxr/merge-pr-python and squashes the following commits: 1ae0013 [Xiangrui Meng] use env default python in merge_spark_pr.py --- 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 7f744d5589ef7..e3ac32ef1a12e 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -1,4 +1,4 @@ -#!/usr/bin/python +#!/usr/bin/env python # # Licensed to the Apache Software Foundation (ASF) under one or more From 5284ca78d17fb4de9a7019f3bbecf86484c13763 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 3 Jun 2014 20:49:14 -0700 Subject: [PATCH 16/54] Enable repartitioning of graph over different number of partitions It is currently very difficult to repartition a graph over a different number of partitions. This PR adds an additional `partitionBy` function that takes the number of partitions. Author: Joseph E. Gonzalez Closes #719 from jegonzal/graph_partitioning_options and squashes the following commits: 730b405 [Joseph E. Gonzalez] adding an additional number of partitions option to partitionBy --- .../src/main/scala/org/apache/spark/graphx/Graph.scala | 10 ++++++++++ .../org/apache/spark/graphx/PartitionStrategy.scala | 8 +++++--- .../scala/org/apache/spark/graphx/impl/GraphImpl.scala | 6 +++++- 3 files changed, 20 insertions(+), 4 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 c4f9d6514cae3..14ae50e6657fd 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -106,9 +106,19 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab /** * Repartitions the edges in the graph according to `partitionStrategy`. + * + * @param the partitioning strategy to use when partitioning the edges in the graph. */ def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] + /** + * Repartitions the edges in the graph according to `partitionStrategy`. + * + * @param the partitioning strategy to use when partitioning the edges in the graph. + * @param numPartitions the number of edge partitions in the new graph. + */ + def partitionBy(partitionStrategy: PartitionStrategy, numPartitions: Int): Graph[VD, ED] + /** * Transforms each vertex attribute in the graph using the map function. * 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 ef412cfd4e6ea..5e7e72a764cc8 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -114,9 +114,11 @@ object PartitionStrategy { */ case object CanonicalRandomVertexCut extends PartitionStrategy { override def getPartition(src: VertexId, dst: VertexId, numParts: PartitionID): PartitionID = { - val lower = math.min(src, dst) - val higher = math.max(src, dst) - math.abs((lower, higher).hashCode()) % numParts + if (src < dst) { + math.abs((src, dst).hashCode()) % numParts + } else { + math.abs((dst, src).hashCode()) % numParts + } } } 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 59d9a8808e56e..15ea05cbe281d 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 @@ -74,7 +74,11 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { - val numPartitions = edges.partitions.size + partitionBy(partitionStrategy, edges.partitions.size) + } + + override def partitionBy( + partitionStrategy: PartitionStrategy, numPartitions: Int): Graph[VD, ED] = { val edTag = classTag[ED] val vdTag = classTag[VD] val newEdges = edges.withPartitionsRDD(edges.map { e => From ab7c62d57300c4033292d06ba840dad02f5552d7 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Tue, 3 Jun 2014 22:33:04 -0700 Subject: [PATCH 17/54] Update spark-ec2 scripts for 1.0.0 on master The change was previously committed only to branch-1.0 as part of https://github.com/apache/spark/commit/a34e6fda1d6fb8e769c21db70845f1a6dde968d8 Author: Aaron Davidson This patch had conflicts when merged, resolved by Committer: Patrick Wendell Closes #938 from aarondav/sparkec2 and squashes the following commits: 067cc31 [Aaron Davidson] Update spark-ec2 scripts for 1.0.0 on master --- 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 8b056f5ea734c..3af9f66e17dc2 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -83,7 +83,7 @@ def parse_args(): "between zones applies)") parser.add_option("-a", "--ami", help="Amazon Machine Image ID to use") parser.add_option( - "-v", "--spark-version", default="0.9.1", + "-v", "--spark-version", default="1.0.0", help="Version of Spark to use: 'X.Y.Z' or a specific git hash") parser.add_option( "--spark-git-repo", @@ -191,7 +191,8 @@ 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", "0.9.1": "0.9.1" + "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", + "1.0.0": "1.0.0" } version = opts.spark_version.replace("v", "") if version not in spark_shark_map: @@ -199,7 +200,6 @@ def get_spark_shark_version(opts): sys.exit(1) return (version, spark_shark_map[version]) - # Attempt to resolve an appropriate AMI given the architecture and # region of the request. def get_spark_ami(opts): From 4ca06256690c5e03058dd179c2fc6437a917cfee Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 3 Jun 2014 22:37:20 -0700 Subject: [PATCH 18/54] SPARK-1806 (addendum) Use non-deprecated methods in Mesos 0.18 The update to Mesos 0.18 caused some deprecation warnings in the build. The change to the non-deprecated version is straightforward as it emulates what the Mesos driver does with the deprecated method anyway (https://github.com/apache/mesos/blob/c5aa1dd22155d79c5a7c33076319299a40fd63b3/src/sched/sched.cpp#L1354) Author: Sean Owen Closes #920 from srowen/SPARK-1806 and squashes the following commits: 8d76b6a [Sean Owen] Use non-deprecated methods in Mesos 0.18 --- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 6 ++++-- .../scheduler/cluster/mesos/MesosSchedulerBackend.scala | 2 +- 2 files changed, 5 insertions(+), 3 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 cbe9bb093d1c9..9f45400bcf852 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 @@ -207,10 +207,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.singleton(offer.getId), Collections.singletonList(task), filters) } else { // Filter it out - d.launchTasks(offer.getId, Collections.emptyList[MesosTaskInfo](), filters) + d.launchTasks( + Collections.singleton(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 f08b19e6782e3..a089a02d42170 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.singleton(offers(i).getId), mesosTasks(i), filters) } } } finally { From c402a4a685721d05932bbc578d997f330ff65a49 Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Tue, 3 Jun 2014 22:47:18 -0700 Subject: [PATCH 19/54] [SPARK-1817] RDD.zip() should verify partition sizes for each partition RDD.zip() will throw an exception if it finds partition sizes are not the same. Author: Kan Zhang Closes #944 from kanzhang/SPARK-1817 and squashes the following commits: c073848 [Kan Zhang] [SPARK-1817] Cosmetic updates 524c670 [Kan Zhang] [SPARK-1817] RDD.zip() should verify partition sizes for each partition --- .../main/scala/org/apache/spark/rdd/RDD.scala | 14 ++- .../org/apache/spark/rdd/ZippedRDD.scala | 87 ------------------- .../org/apache/spark/CheckpointSuite.scala | 26 +++--- .../scala/org/apache/spark/rdd/RDDSuite.scala | 4 + project/MimaExcludes.scala | 2 + 5 files changed, 33 insertions(+), 100 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala 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 585b2f76afa65..54bdc3e7cbc7a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -654,7 +654,19 @@ abstract class RDD[T: ClassTag]( * partitions* and the *same number of elements in each partition* (e.g. one was made through * a map on the other). */ - def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = new ZippedRDD(sc, this, other) + def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = { + zipPartitions(other, true) { (thisIter, otherIter) => + new Iterator[(T, U)] { + def hasNext = (thisIter.hasNext, otherIter.hasNext) match { + case (true, true) => true + case (false, false) => false + case _ => throw new SparkException("Can only zip RDDs with " + + "same number of elements in each partition") + } + def next = (thisIter.next, otherIter.next) + } + } + } /** * Zip this RDD's partitions with one (or more) RDD(s) and return a new RDD by diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala deleted file mode 100644 index b8110ffc42f2d..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala +++ /dev/null @@ -1,87 +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 java.io.{IOException, ObjectOutputStream} - -import scala.reflect.ClassTag - -import org.apache.spark.{OneToOneDependency, Partition, SparkContext, TaskContext} - -private[spark] class ZippedPartition[T: ClassTag, U: ClassTag]( - idx: Int, - @transient rdd1: RDD[T], - @transient rdd2: RDD[U] - ) extends Partition { - - var partition1 = rdd1.partitions(idx) - var partition2 = rdd2.partitions(idx) - override val index: Int = idx - - def partitions = (partition1, partition2) - - @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { - // Update the reference to parent partition at the time of task serialization - partition1 = rdd1.partitions(idx) - partition2 = rdd2.partitions(idx) - oos.defaultWriteObject() - } -} - -private[spark] class ZippedRDD[T: ClassTag, U: ClassTag]( - sc: SparkContext, - var rdd1: RDD[T], - var rdd2: RDD[U]) - extends RDD[(T, U)](sc, List(new OneToOneDependency(rdd1), new OneToOneDependency(rdd2))) { - - override def getPartitions: Array[Partition] = { - if (rdd1.partitions.size != rdd2.partitions.size) { - throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions") - } - val array = new Array[Partition](rdd1.partitions.size) - for (i <- 0 until rdd1.partitions.size) { - array(i) = new ZippedPartition(i, rdd1, rdd2) - } - array - } - - override def compute(s: Partition, context: TaskContext): Iterator[(T, U)] = { - val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions - rdd1.iterator(partition1, context).zip(rdd2.iterator(partition2, context)) - } - - override def getPreferredLocations(s: Partition): Seq[String] = { - val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions - val pref1 = rdd1.preferredLocations(partition1) - val pref2 = rdd2.preferredLocations(partition2) - // Check whether there are any hosts that match both RDDs; otherwise return the union - val exactMatchLocations = pref1.intersect(pref2) - if (!exactMatchLocations.isEmpty) { - exactMatchLocations - } else { - (pref1 ++ pref2).distinct - } - } - - override def clearDependencies() { - super.clearDependencies() - rdd1 = null - rdd2 = null - } -} diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 64933f4b1046d..f64f3c9036034 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -167,26 +167,28 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { }) } - test("ZippedRDD") { - testRDD(rdd => new ZippedRDD(sc, rdd, rdd.map(x => x))) - testRDDPartitions(rdd => new ZippedRDD(sc, rdd, rdd.map(x => x))) + test("ZippedPartitionsRDD") { + testRDD(rdd => rdd.zip(rdd.map(x => x))) + testRDDPartitions(rdd => rdd.zip(rdd.map(x => x))) - // Test that the ZippedPartition updates parent partitions - // after the parent RDD has been checkpointed and parent partitions have been changed. - // Note that this test is very specific to the current implementation of ZippedRDD. + // Test that ZippedPartitionsRDD updates parent partitions after parent RDDs have + // been checkpointed and parent partitions have been changed. + // Note that this test is very specific to the implementation of ZippedPartitionsRDD. val rdd = generateFatRDD() - val zippedRDD = new ZippedRDD(sc, rdd, rdd.map(x => x)) + val zippedRDD = rdd.zip(rdd.map(x => x)).asInstanceOf[ZippedPartitionsRDD2[_, _, _]] zippedRDD.rdd1.checkpoint() zippedRDD.rdd2.checkpoint() val partitionBeforeCheckpoint = - serializeDeserialize(zippedRDD.partitions.head.asInstanceOf[ZippedPartition[_, _]]) + serializeDeserialize(zippedRDD.partitions.head.asInstanceOf[ZippedPartitionsPartition]) zippedRDD.count() val partitionAfterCheckpoint = - serializeDeserialize(zippedRDD.partitions.head.asInstanceOf[ZippedPartition[_, _]]) + serializeDeserialize(zippedRDD.partitions.head.asInstanceOf[ZippedPartitionsPartition]) assert( - partitionAfterCheckpoint.partition1.getClass != partitionBeforeCheckpoint.partition1.getClass && - partitionAfterCheckpoint.partition2.getClass != partitionBeforeCheckpoint.partition2.getClass, - "ZippedRDD.partition1 and ZippedRDD.partition2 not updated after parent RDD is checkpointed" + partitionAfterCheckpoint.partitions(0).getClass != + partitionBeforeCheckpoint.partitions(0).getClass && + partitionAfterCheckpoint.partitions(1).getClass != + partitionBeforeCheckpoint.partitions(1).getClass, + "ZippedPartitionsRDD partition 0 (or 1) not updated after parent RDDs are checkpointed" ) } 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 bbd0c14178368..286e221e33b78 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -350,6 +350,10 @@ class RDDSuite extends FunSuite with SharedSparkContext { intercept[IllegalArgumentException] { nums.zip(sc.parallelize(1 to 4, 1)).collect() } + + intercept[SparkException] { + nums.zip(sc.parallelize(1 to 5, 2)).collect() + } } test("partition pruning") { diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index fadf6a4d8bb4a..dd7efceb23c96 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -54,6 +54,8 @@ object MimaExcludes { ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.api.java.JavaDoubleRDD.countApproxDistinct$default$1") ) ++ + MimaBuild.excludeSparkClass("rdd.ZippedRDD") ++ + MimaBuild.excludeSparkClass("rdd.ZippedPartition") ++ MimaBuild.excludeSparkClass("util.SerializableHyperLogLog") case v if v.startsWith("1.0") => Seq( From b8d25800393d0208a76813bcd94509ac24a3add5 Mon Sep 17 00:00:00 2001 From: Neville Li Date: Wed, 4 Jun 2014 01:51:34 -0700 Subject: [PATCH 20/54] [MLLIB] set RDD names in ALS This is very useful when debugging & fine tuning jobs with large data sets. Author: Neville Li Closes #966 from nevillelyh/master and squashes the following commits: 6747764 [Neville Li] [MLLIB] use string interpolation for RDD names 3b15d34 [Neville Li] [MLLIB] set RDD names in ALS --- .../apache/spark/mllib/recommendation/ALS.scala | 16 +++++++++++----- 1 file changed, 11 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 cfc3b6860649a..d743bd7dd1825 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 @@ -201,6 +201,10 @@ class ALS private ( val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock, partitioner) val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock, partitioner) + userInLinks.setName("userInLinks") + userOutLinks.setName("userOutLinks") + productInLinks.setName("productInLinks") + productOutLinks.setName("productOutLinks") // Initialize user and product factors randomly, but use a deterministic seed for each // partition so that fault recovery works @@ -225,14 +229,14 @@ class ALS private ( // perform ALS update logInfo("Re-computing I given U (Iteration %d/%d)".format(iter, iterations)) // Persist users because it will be called twice. - users.persist() + users.setName(s"users-$iter").persist() val YtY = Some(sc.broadcast(computeYtY(users))) val previousProducts = products products = updateFeatures(users, userOutLinks, productInLinks, partitioner, rank, lambda, alpha, YtY) previousProducts.unpersist() logInfo("Re-computing U given I (Iteration %d/%d)".format(iter, iterations)) - products.persist() + products.setName(s"products-$iter").persist() val XtX = Some(sc.broadcast(computeYtY(products))) val previousUsers = users users = updateFeatures(products, productOutLinks, userInLinks, partitioner, rank, lambda, @@ -245,22 +249,24 @@ class ALS private ( logInfo("Re-computing I given U (Iteration %d/%d)".format(iter, iterations)) products = updateFeatures(users, userOutLinks, productInLinks, partitioner, rank, lambda, alpha, YtY = None) + products.setName(s"products-$iter") logInfo("Re-computing U given I (Iteration %d/%d)".format(iter, iterations)) users = updateFeatures(products, productOutLinks, userInLinks, partitioner, rank, lambda, alpha, YtY = None) + users.setName(s"users-$iter") } } // The last `products` will be used twice. One to generate the last `users` and the other to // generate `productsOut`. So we cache it for better performance. - products.persist() + products.setName("products").persist() // Flatten and cache the two final RDDs to un-block them val usersOut = unblockFactors(users, userOutLinks) val productsOut = unblockFactors(products, productOutLinks) - usersOut.persist() - productsOut.persist() + usersOut.setName("usersOut").persist() + productsOut.setName("productsOut").persist() // Materialize usersOut and productsOut. usersOut.count() From d341b17c2a0a4fce04045e13fb4a3b0621296320 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 4 Jun 2014 11:27:08 -0700 Subject: [PATCH 21/54] SPARK-1973. Add randomSplit to JavaRDD (with tests, and tidy Java tests) I'd like to use randomSplit through the Java API, and would like to add a convenience wrapper for this method to JavaRDD. This is fairly trivial. (In fact, is the intent that JavaRDD not wrap every RDD method? and that sometimes users should just use JavaRDD.wrapRDD()?) Along the way, I added tests for it, and also touched up the Java API test style and behavior. This is maybe the more useful part of this small change. Author: Sean Owen Author: Xiangrui Meng This patch had conflicts when merged, resolved by Committer: Xiangrui Meng Closes #919 from srowen/SPARK-1973 and squashes the following commits: 148cb7b [Sean Owen] Some final Java test polish, while we are at it 1fc3f3e [Xiangrui Meng] more cleaning on Java 8 tests 9ebc57f [Sean Owen] Use accumulator instead of temp files to test foreach 5efb0be [Sean Owen] Add Java randomSplit, and unit tests (including for sample) 5dcc158 [Sean Owen] Simplified Java 8 test with new language features, and fixed the name of MLB's greatest team 91a1769 [Sean Owen] Touch up minor style issues in existing Java API suite test --- .../org/apache/spark/api/java/JavaRDD.scala | 22 + .../java/org/apache/spark/JavaAPISuite.java | 193 ++++----- .../java/org/apache/spark/Java8APISuite.java | 96 +++-- .../apache/spark/streaming/Java8APISuite.java | 381 +++++++++--------- 4 files changed, 358 insertions(+), 334 deletions(-) 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 dc698dea75e43..23d13710794af 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 @@ -108,6 +108,28 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaRDD[T] = wrapRDD(rdd.sample(withReplacement, fraction, seed)) + + /** + * Randomly splits this RDD with the provided weights. + * + * @param weights weights for splits, will be normalized if they don't sum to 1 + * + * @return split RDDs in an array + */ + def randomSplit(weights: Array[Double]): Array[JavaRDD[T]] = + randomSplit(weights, Utils.random.nextLong) + + /** + * 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 + * + * @return split RDDs in an array + */ + def randomSplit(weights: Array[Double], seed: Long): Array[JavaRDD[T]] = + rdd.randomSplit(weights, seed).map(wrapRDD) + /** * Return the union of this RDD and another one. Any identical elements will appear multiple * times (use `.distinct()` to eliminate them). diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index b78309f81cb8c..50a62129116f1 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -23,6 +23,7 @@ import scala.Tuple2; import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.base.Optional; import com.google.common.base.Charsets; @@ -48,7 +49,6 @@ 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; @@ -70,16 +70,6 @@ public void tearDown() { sc = null; } - static class ReverseIntComparator implements Comparator, Serializable { - - @Override - public int compare(Integer a, Integer b) { - if (a > b) return -1; - else if (a < b) return 1; - else return 0; - } - } - @SuppressWarnings("unchecked") @Test public void sparkContextUnion() { @@ -124,7 +114,7 @@ public void intersection() { JavaRDD intersections = s1.intersection(s2); Assert.assertEquals(3, intersections.count()); - ArrayList list = new ArrayList(); + List list = new ArrayList(); JavaRDD empty = sc.parallelize(list); JavaRDD emptyIntersection = empty.intersection(s2); Assert.assertEquals(0, emptyIntersection.count()); @@ -144,6 +134,28 @@ public void intersection() { Assert.assertEquals(2, pIntersection.count()); } + @Test + public void sample() { + List ints = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + JavaRDD rdd = sc.parallelize(ints); + JavaRDD sample20 = rdd.sample(true, 0.2, 11); + // expected 2 but of course result varies randomly a bit + Assert.assertEquals(3, sample20.count()); + JavaRDD sample20NoReplacement = rdd.sample(false, 0.2, 11); + Assert.assertEquals(2, sample20NoReplacement.count()); + } + + @Test + public void randomSplit() { + List ints = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + JavaRDD rdd = sc.parallelize(ints); + JavaRDD[] splits = rdd.randomSplit(new double[] { 0.4, 0.6, 1.0 }, 11); + Assert.assertEquals(3, splits.length); + Assert.assertEquals(2, splits[0].count()); + Assert.assertEquals(3, splits[1].count()); + Assert.assertEquals(5, splits[2].count()); + } + @Test public void sortByKey() { List> pairs = new ArrayList>(); @@ -161,26 +173,24 @@ public void sortByKey() { Assert.assertEquals(new Tuple2(3, 2), sortedPairs.get(2)); // Custom comparator - sortedRDD = rdd.sortByKey(new ReverseIntComparator(), false); + sortedRDD = rdd.sortByKey(Collections.reverseOrder(), false); Assert.assertEquals(new Tuple2(-1, 1), sortedRDD.first()); sortedPairs = sortedRDD.collect(); Assert.assertEquals(new Tuple2(0, 4), sortedPairs.get(1)); Assert.assertEquals(new Tuple2(3, 2), sortedPairs.get(2)); } - static int foreachCalls = 0; - @Test public void foreach() { - foreachCalls = 0; + final Accumulator accum = sc.accumulator(0); JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); rdd.foreach(new VoidFunction() { @Override - public void call(String s) { - foreachCalls++; + public void call(String s) throws IOException { + accum.add(1); } }); - Assert.assertEquals(2, foreachCalls); + Assert.assertEquals(2, accum.value().intValue()); } @Test @@ -188,7 +198,7 @@ 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)); + Assert.assertEquals(correct, result); } @Test @@ -196,7 +206,7 @@ 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); + Assert.assertEquals(4, new HashSet(indexes.collect()).size()); } @Test @@ -205,7 +215,7 @@ public void zipWithIndex() { JavaPairRDD zip = sc.parallelize(dataArray).zipWithIndex(); JavaRDD indexes = zip.values(); List correctIndexes = Arrays.asList(0L, 1L, 2L, 3L); - Assert.assertTrue(indexes.collect().equals(correctIndexes)); + Assert.assertEquals(correctIndexes, indexes.collect()); } @SuppressWarnings("unchecked") @@ -252,8 +262,10 @@ public void cogroup() { new Tuple2("Oranges", 2), new Tuple2("Apples", 3) )); - JavaPairRDD, Iterable>> cogrouped = categories.cogroup(prices); - Assert.assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + 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(); @@ -281,8 +293,7 @@ public void leftOuterJoin() { rdd1.leftOuterJoin(rdd2).filter( new Function>>, Boolean>() { @Override - public Boolean call(Tuple2>> tup) - throws Exception { + public Boolean call(Tuple2>> tup) { return !tup._2()._2().isPresent(); } }).first(); @@ -356,8 +367,7 @@ public Integer call(Integer a, Integer b) { Assert.assertEquals(2, localCounts.get(2).intValue()); Assert.assertEquals(3, localCounts.get(3).intValue()); - localCounts = rdd.reduceByKeyLocally(new Function2() { + localCounts = rdd.reduceByKeyLocally(new Function2() { @Override public Integer call(Integer a, Integer b) { return a + b; @@ -448,16 +458,17 @@ public void map() { JavaDoubleRDD doubles = rdd.mapToDouble(new DoubleFunction() { @Override public double call(Integer x) { - return 1.0 * x; + return x.doubleValue(); } }).cache(); doubles.collect(); - JavaPairRDD pairs = rdd.mapToPair(new PairFunction() { - @Override - public Tuple2 call(Integer x) { - return new Tuple2(x, x); - } - }).cache(); + JavaPairRDD pairs = rdd.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(Integer x) { + return new Tuple2(x, x); + } + }).cache(); pairs.collect(); JavaRDD strings = rdd.map(new Function() { @Override @@ -487,7 +498,9 @@ public Iterable call(String x) { @Override public Iterable> call(String s) { List> pairs = new LinkedList>(); - for (String word : s.split(" ")) pairs.add(new Tuple2(word, word)); + for (String word : s.split(" ")) { + pairs.add(new Tuple2(word, word)); + } return pairs; } } @@ -499,7 +512,9 @@ public Iterable> call(String s) { @Override public Iterable call(String s) { List lengths = new LinkedList(); - for (String word : s.split(" ")) lengths.add(word.length() * 1.0); + for (String word : s.split(" ")) { + lengths.add((double) word.length()); + } return lengths; } }); @@ -521,7 +536,7 @@ public void mapsFromPairsToPairs() { JavaPairRDD swapped = pairRDD.flatMapToPair( new PairFlatMapFunction, String, Integer>() { @Override - public Iterable> call(Tuple2 item) throws Exception { + public Iterable> call(Tuple2 item) { return Collections.singletonList(item.swap()); } }); @@ -530,7 +545,7 @@ public Iterable> call(Tuple2 item) thro // There was never a bug here, but it's worth testing: pairRDD.mapToPair(new PairFunction, String, Integer>() { @Override - public Tuple2 call(Tuple2 item) throws Exception { + public Tuple2 call(Tuple2 item) { return item.swap(); } }).collect(); @@ -631,14 +646,10 @@ public void wholeTextFiles() throws IOException { byte[] content2 = "spark is also easy to use.\n".getBytes("utf-8"); String tempDirName = tempDir.getAbsolutePath(); - DataOutputStream ds = new DataOutputStream(new FileOutputStream(tempDirName + "/part-00000")); - ds.write(content1); - ds.close(); - ds = new DataOutputStream(new FileOutputStream(tempDirName + "/part-00001")); - ds.write(content2); - ds.close(); - - HashMap container = new HashMap(); + Files.write(content1, new File(tempDirName + "/part-00000")); + Files.write(content2, new File(tempDirName + "/part-00001")); + + Map container = new HashMap(); container.put(tempDirName+"/part-00000", new Text(content1).toString()); container.put(tempDirName+"/part-00001", new Text(content2).toString()); @@ -844,7 +855,7 @@ public void zip() { JavaDoubleRDD doubles = rdd.mapToDouble(new DoubleFunction() { @Override public double call(Integer x) { - return 1.0 * x; + return x.doubleValue(); } }); JavaPairRDD zipped = rdd.zip(doubles); @@ -859,17 +870,7 @@ public void zipPartitions() { new FlatMapFunction2, Iterator, Integer>() { @Override public Iterable call(Iterator i, Iterator s) { - int sizeI = 0; - int sizeS = 0; - while (i.hasNext()) { - sizeI += 1; - i.next(); - } - while (s.hasNext()) { - sizeS += 1; - s.next(); - } - return Arrays.asList(sizeI, sizeS); + return Arrays.asList(Iterators.size(i), Iterators.size(s)); } }; @@ -883,6 +884,7 @@ public void accumulators() { final Accumulator intAccum = sc.intAccumulator(10); rdd.foreach(new VoidFunction() { + @Override public void call(Integer x) { intAccum.add(x); } @@ -891,6 +893,7 @@ public void call(Integer x) { final Accumulator doubleAccum = sc.doubleAccumulator(10.0); rdd.foreach(new VoidFunction() { + @Override public void call(Integer x) { doubleAccum.add((double) x); } @@ -899,14 +902,17 @@ public void call(Integer x) { // Try a custom accumulator type AccumulatorParam floatAccumulatorParam = new AccumulatorParam() { + @Override public Float addInPlace(Float r, Float t) { return r + t; } + @Override public Float addAccumulator(Float r, Float t) { return r + t; } + @Override public Float zero(Float initialValue) { return 0.0f; } @@ -914,6 +920,7 @@ public Float zero(Float initialValue) { final Accumulator floatAccum = sc.accumulator(10.0f, floatAccumulatorParam); rdd.foreach(new VoidFunction() { + @Override public void call(Integer x) { floatAccum.add((float) x); } @@ -929,7 +936,8 @@ public void call(Integer x) { public void keyBy() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2)); List> s = rdd.keyBy(new Function() { - public String call(Integer t) throws Exception { + @Override + public String call(Integer t) { return t.toString(); } }).collect(); @@ -941,10 +949,10 @@ public String call(Integer t) throws Exception { public void checkpointAndComputation() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); sc.setCheckpointDir(tempDir.getAbsolutePath()); - Assert.assertEquals(false, rdd.isCheckpointed()); + Assert.assertFalse(rdd.isCheckpointed()); rdd.checkpoint(); rdd.count(); // Forces the DAG to cause a checkpoint - Assert.assertEquals(true, rdd.isCheckpointed()); + Assert.assertTrue(rdd.isCheckpointed()); Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), rdd.collect()); } @@ -952,10 +960,10 @@ public void checkpointAndComputation() { public void checkpointAndRestore() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); sc.setCheckpointDir(tempDir.getAbsolutePath()); - Assert.assertEquals(false, rdd.isCheckpointed()); + Assert.assertFalse(rdd.isCheckpointed()); rdd.checkpoint(); rdd.count(); // Forces the DAG to cause a checkpoint - Assert.assertEquals(true, rdd.isCheckpointed()); + Assert.assertTrue(rdd.isCheckpointed()); Assert.assertTrue(rdd.getCheckpointFile().isPresent()); JavaRDD recovered = sc.checkpointFile(rdd.getCheckpointFile().get()); @@ -966,16 +974,17 @@ public void checkpointAndRestore() { @Test public void mapOnPairRDD() { JavaRDD rdd1 = sc.parallelize(Arrays.asList(1,2,3,4)); - JavaPairRDD rdd2 = rdd1.mapToPair(new PairFunction() { - @Override - public Tuple2 call(Integer i) throws Exception { - return new Tuple2(i, i % 2); - } - }); + JavaPairRDD rdd2 = rdd1.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(Integer i) { + return new Tuple2(i, i % 2); + } + }); JavaPairRDD rdd3 = rdd2.mapToPair( new PairFunction, Integer, Integer>() { @Override - public Tuple2 call(Tuple2 in) throws Exception { + public Tuple2 call(Tuple2 in) { return new Tuple2(in._2(), in._1()); } }); @@ -992,14 +1001,15 @@ public Tuple2 call(Tuple2 in) throws Excepti public void collectPartitions() { JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3); - JavaPairRDD rdd2 = rdd1.mapToPair(new PairFunction() { - @Override - public Tuple2 call(Integer i) throws Exception { - return new Tuple2(i, i % 2); - } - }); + JavaPairRDD rdd2 = rdd1.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(Integer i) { + return new Tuple2(i, i % 2); + } + }); - List[] parts = rdd1.collectPartitions(new int[] {0}); + List[] parts = rdd1.collectPartitions(new int[] {0}); Assert.assertEquals(Arrays.asList(1, 2), parts[0]); parts = rdd1.collectPartitions(new int[] {1, 2}); @@ -1010,14 +1020,14 @@ public Tuple2 call(Integer i) throws Exception { new Tuple2(2, 0)), rdd2.collectPartitions(new int[] {0})[0]); - parts = rdd2.collectPartitions(new int[] {1, 2}); + List>[] parts2 = rdd2.collectPartitions(new int[] {1, 2}); Assert.assertEquals(Arrays.asList(new Tuple2(3, 1), new Tuple2(4, 0)), - parts[0]); + parts2[0]); Assert.assertEquals(Arrays.asList(new Tuple2(5, 1), new Tuple2(6, 0), new Tuple2(7, 1)), - parts[1]); + parts2[1]); } @Test @@ -1034,10 +1044,12 @@ public void countApproxDistinct() { @Test public void countApproxDistinctByKey() { List> arrayData = new ArrayList>(); - for (int i = 10; i < 100; i++) - for (int j = 0; j < i; j++) + for (int i = 10; i < 100; i++) { + for (int j = 0; j < i; j++) { arrayData.add(new Tuple2(i, j)); - + } + } + double relativeSD = 0.001; JavaPairRDD pairRdd = sc.parallelizePairs(arrayData); List> res = pairRdd.countApproxDistinctByKey(8, 0).collect(); for (Tuple2 resItem : res) { @@ -1053,12 +1065,13 @@ public void countApproxDistinctByKey() { public void collectAsMapWithIntArrayValues() { // Regression test for SPARK-1040 JavaRDD rdd = sc.parallelize(Arrays.asList(1)); - JavaPairRDD pairRDD = rdd.mapToPair(new PairFunction() { - @Override - public Tuple2 call(Integer x) throws Exception { - return new Tuple2(x, new int[] { x }); - } - }); + JavaPairRDD pairRDD = rdd.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(Integer x) { + return new Tuple2(x, new int[] { x }); + } + }); pairRDD.collect(); // Works fine pairRDD.collectAsMap(); // Used to crash with ClassCastException } 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 c366c10b15a20..729bc0459ce52 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 @@ -99,16 +99,16 @@ public void groupBy() { @Test public void leftOuterJoin() { JavaPairRDD rdd1 = sc.parallelizePairs(Arrays.asList( - new Tuple2(1, 1), - new Tuple2(1, 2), - new Tuple2(2, 1), - new Tuple2(3, 1) + new Tuple2<>(1, 1), + new Tuple2<>(1, 2), + new Tuple2<>(2, 1), + new Tuple2<>(3, 1) )); JavaPairRDD rdd2 = sc.parallelizePairs(Arrays.asList( - new Tuple2(1, 'x'), - new Tuple2(2, 'y'), - new Tuple2(2, 'z'), - new Tuple2(4, 'w') + new Tuple2<>(1, 'x'), + new Tuple2<>(2, 'y'), + new Tuple2<>(2, 'z'), + new Tuple2<>(4, 'w') )); List>>> joined = rdd1.leftOuterJoin(rdd2).collect(); @@ -133,11 +133,11 @@ public void foldReduce() { @Test public void foldByKey() { List> pairs = Arrays.asList( - new Tuple2(2, 1), - new Tuple2(2, 1), - new Tuple2(1, 1), - new Tuple2(3, 2), - new Tuple2(3, 1) + new Tuple2<>(2, 1), + new Tuple2<>(2, 1), + new Tuple2<>(1, 1), + new Tuple2<>(3, 2), + new Tuple2<>(3, 1) ); JavaPairRDD rdd = sc.parallelizePairs(pairs); JavaPairRDD sums = rdd.foldByKey(0, (a, b) -> a + b); @@ -149,11 +149,11 @@ public void foldByKey() { @Test public void reduceByKey() { List> pairs = Arrays.asList( - new Tuple2(2, 1), - new Tuple2(2, 1), - new Tuple2(1, 1), - new Tuple2(3, 2), - new Tuple2(3, 1) + new Tuple2<>(2, 1), + new Tuple2<>(2, 1), + new Tuple2<>(1, 1), + new Tuple2<>(3, 2), + new Tuple2<>(3, 1) ); JavaPairRDD rdd = sc.parallelizePairs(pairs); JavaPairRDD counts = rdd.reduceByKey((a, b) -> a + b); @@ -177,7 +177,7 @@ public void map() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); JavaDoubleRDD doubles = rdd.mapToDouble(x -> 1.0 * x).cache(); doubles.collect(); - JavaPairRDD pairs = rdd.mapToPair(x -> new Tuple2(x, x)) + JavaPairRDD pairs = rdd.mapToPair(x -> new Tuple2<>(x, x)) .cache(); pairs.collect(); JavaRDD strings = rdd.map(x -> x.toString()).cache(); @@ -194,31 +194,31 @@ public void flatMap() { Assert.assertEquals(11, words.count()); JavaPairRDD pairs = rdd.flatMapToPair(s -> { - List> pairs2 = new LinkedList>(); - for (String word : s.split(" ")) pairs2.add(new Tuple2(word, word)); + List> pairs2 = new LinkedList<>(); + for (String word : s.split(" ")) pairs2.add(new Tuple2<>(word, word)); return pairs2; }); - Assert.assertEquals(new Tuple2("Hello", "Hello"), pairs.first()); + Assert.assertEquals(new Tuple2<>("Hello", "Hello"), pairs.first()); Assert.assertEquals(11, pairs.count()); JavaDoubleRDD doubles = rdd.flatMapToDouble(s -> { - List lengths = new LinkedList(); + List lengths = new LinkedList<>(); for (String word : s.split(" ")) lengths.add(word.length() * 1.0); return lengths; }); Double x = doubles.first(); - Assert.assertEquals(5.0, doubles.first().doubleValue(), 0.01); + Assert.assertEquals(5.0, doubles.first(), 0.01); Assert.assertEquals(11, pairs.count()); } @Test public void mapsFromPairsToPairs() { List> pairs = Arrays.asList( - new Tuple2(1, "a"), - new Tuple2(2, "aa"), - new Tuple2(3, "aaa") + new Tuple2<>(1, "a"), + new Tuple2<>(2, "aa"), + new Tuple2<>(3, "aaa") ); JavaPairRDD pairRDD = sc.parallelizePairs(pairs); @@ -251,19 +251,18 @@ public void sequenceFile() { tempDir.deleteOnExit(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( - new Tuple2(1, "a"), - new Tuple2(2, "aa"), - new Tuple2(3, "aaa") + new Tuple2<>(1, "a"), + new Tuple2<>(2, "aa"), + new Tuple2<>(3, "aaa") ); JavaPairRDD rdd = sc.parallelizePairs(pairs); - rdd.mapToPair(pair -> - new Tuple2(new IntWritable(pair._1()), new Text(pair._2()))) + rdd.mapToPair(pair -> new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()))) .saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); // Try reading the output back as an object file JavaPairRDD readRDD = sc.sequenceFile(outputDir, IntWritable.class, Text.class) - .mapToPair(pair -> new Tuple2(pair._1().get(), pair._2().toString())); + .mapToPair(pair -> new Tuple2<>(pair._1().get(), pair._2().toString())); Assert.assertEquals(pairs, readRDD.collect()); Utils.deleteRecursively(tempDir); } @@ -325,7 +324,7 @@ public Float zero(Float initialValue) { } }; - final Accumulator floatAccum = sc.accumulator((Float) 10.0f, floatAccumulatorParam); + final Accumulator floatAccum = sc.accumulator(10.0f, floatAccumulatorParam); rdd.foreach(x -> floatAccum.add((float) x)); Assert.assertEquals((Float) 25.0f, floatAccum.value()); @@ -338,22 +337,22 @@ public Float zero(Float initialValue) { public void keyBy() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2)); List> s = rdd.keyBy(x -> x.toString()).collect(); - Assert.assertEquals(new Tuple2("1", 1), s.get(0)); - Assert.assertEquals(new Tuple2("2", 2), s.get(1)); + Assert.assertEquals(new Tuple2<>("1", 1), s.get(0)); + Assert.assertEquals(new Tuple2<>("2", 2), s.get(1)); } @Test public void mapOnPairRDD() { JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4)); JavaPairRDD rdd2 = - rdd1.mapToPair(i -> new Tuple2(i, i % 2)); + rdd1.mapToPair(i -> new Tuple2<>(i, i % 2)); JavaPairRDD rdd3 = - rdd2.mapToPair(in -> new Tuple2(in._2(), in._1())); + rdd2.mapToPair(in -> new Tuple2<>(in._2(), in._1())); Assert.assertEquals(Arrays.asList( new Tuple2(1, 1), - new Tuple2(0, 2), - new Tuple2(1, 3), - new Tuple2(0, 4)), rdd3.collect()); + new Tuple2<>(0, 2), + new Tuple2<>(1, 3), + new Tuple2<>(0, 4)), rdd3.collect()); } @Test @@ -361,7 +360,7 @@ public void collectPartitions() { JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3); JavaPairRDD rdd2 = - rdd1.mapToPair(i -> new Tuple2(i, i % 2)); + rdd1.mapToPair(i -> new Tuple2<>(i, i % 2)); List[] parts = rdd1.collectPartitions(new int[]{0}); Assert.assertEquals(Arrays.asList(1, 2), parts[0]); @@ -369,16 +368,13 @@ public void collectPartitions() { Assert.assertEquals(Arrays.asList(3, 4), parts[0]); Assert.assertEquals(Arrays.asList(5, 6, 7), parts[1]); - Assert.assertEquals(Arrays.asList(new Tuple2(1, 1), - new Tuple2(2, 0)), + Assert.assertEquals(Arrays.asList(new Tuple2<>(1, 1), new Tuple2<>(2, 0)), rdd2.collectPartitions(new int[]{0})[0]); parts = rdd2.collectPartitions(new int[]{1, 2}); - Assert.assertEquals(Arrays.asList(new Tuple2(3, 1), - new Tuple2(4, 0)), parts[0]); - Assert.assertEquals(Arrays.asList(new Tuple2(5, 1), - new Tuple2(6, 0), - new Tuple2(7, 1)), parts[1]); + Assert.assertEquals(Arrays.asList(new Tuple2<>(3, 1), new Tuple2<>(4, 0)), parts[0]); + Assert.assertEquals(Arrays.asList(new Tuple2<>(5, 1), new Tuple2<>(6, 0), new Tuple2<>(7, 1)), + parts[1]); } @Test @@ -386,7 +382,7 @@ public void collectAsMapWithIntArrayValues() { // Regression test for SPARK-1040 JavaRDD rdd = sc.parallelize(Arrays.asList(new Integer[]{1})); JavaPairRDD pairRDD = - rdd.mapToPair(x -> new Tuple2(x, new int[]{x})); + rdd.mapToPair(x -> new Tuple2<>(x, new int[]{x})); pairRDD.collect(); // Works fine Map map = pairRDD.collectAsMap(); // Used to crash with ClassCastException } diff --git a/extras/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java index 43df0dea614bc..73091cfe2c09e 100644 --- a/extras/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java +++ b/extras/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java @@ -39,6 +39,7 @@ * Most of these tests replicate org.apache.spark.streaming.JavaAPISuite using java 8 * lambda syntax. */ +@SuppressWarnings("unchecked") public class Java8APISuite extends LocalJavaStreamingContext implements Serializable { @Test @@ -52,7 +53,7 @@ public void testMap() { Arrays.asList(9, 4)); JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream letterCount = stream.map(s -> s.length()); + JavaDStream letterCount = stream.map(String::length); JavaTestUtils.attachTestOutputStream(letterCount); List> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -63,7 +64,7 @@ public void testMap() { public void testFilter() { List> inputData = Arrays.asList( Arrays.asList("giants", "dodgers"), - Arrays.asList("yankees", "red socks")); + Arrays.asList("yankees", "red sox")); List> expected = Arrays.asList( Arrays.asList("giants"), @@ -81,11 +82,11 @@ public void testFilter() { public void testMapPartitions() { List> inputData = Arrays.asList( Arrays.asList("giants", "dodgers"), - Arrays.asList("yankees", "red socks")); + Arrays.asList("yankees", "red sox")); List> expected = Arrays.asList( Arrays.asList("GIANTSDODGERS"), - Arrays.asList("YANKEESRED SOCKS")); + Arrays.asList("YANKEESRED SOX")); JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaDStream mapped = stream.mapPartitions(in -> { @@ -172,7 +173,7 @@ public void testVariousTransform() { JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); List>> pairInputData = - Arrays.asList(Arrays.asList(new Tuple2("x", 1))); + Arrays.asList(Arrays.asList(new Tuple2<>("x", 1))); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream( JavaTestUtils.attachTestInputStream(ssc, pairInputData, 1)); @@ -192,32 +193,32 @@ public void testVariousTransform() { public void testTransformWith() { List>> stringStringKVStream1 = Arrays.asList( Arrays.asList( - new Tuple2("california", "dodgers"), - new Tuple2("new york", "yankees")), + new Tuple2<>("california", "dodgers"), + new Tuple2<>("new york", "yankees")), Arrays.asList( - new Tuple2("california", "sharks"), - new Tuple2("new york", "rangers"))); + new Tuple2<>("california", "sharks"), + new Tuple2<>("new york", "rangers"))); List>> stringStringKVStream2 = Arrays.asList( Arrays.asList( - new Tuple2("california", "giants"), - new Tuple2("new york", "mets")), + new Tuple2<>("california", "giants"), + new Tuple2<>("new york", "mets")), Arrays.asList( - new Tuple2("california", "ducks"), - new Tuple2("new york", "islanders"))); + new Tuple2<>("california", "ducks"), + new Tuple2<>("new york", "islanders"))); - List>>> expected = Arrays.asList( + List>>> expected = Arrays.asList( Sets.newHashSet( - new Tuple2>("california", - new Tuple2("dodgers", "giants")), - new Tuple2>("new york", - new Tuple2("yankees", "mets"))), + new Tuple2<>("california", + new Tuple2<>("dodgers", "giants")), + new Tuple2<>("new york", + new Tuple2<>("yankees", "mets"))), Sets.newHashSet( - new Tuple2>("california", - new Tuple2("sharks", "ducks")), - new Tuple2>("new york", - new Tuple2("rangers", "islanders")))); + new Tuple2<>("california", + new Tuple2<>("sharks", "ducks")), + new Tuple2<>("new york", + new Tuple2<>("rangers", "islanders")))); JavaDStream> stream1 = JavaTestUtils.attachTestInputStream( ssc, stringStringKVStream1, 1); @@ -232,7 +233,7 @@ public void testTransformWith() { JavaTestUtils.attachTestOutputStream(joined); List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); - List>>> unorderedResult = Lists.newArrayList(); + List>>> unorderedResult = Lists.newArrayList(); for (List>> res : result) { unorderedResult.add(Sets.newHashSet(res)); } @@ -251,9 +252,9 @@ public void testVariousTransformWith() { JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 1); List>> pairInputData1 = - Arrays.asList(Arrays.asList(new Tuple2("x", 1))); + Arrays.asList(Arrays.asList(new Tuple2<>("x", 1))); List>> pairInputData2 = - Arrays.asList(Arrays.asList(new Tuple2(1.0, 'x'))); + Arrays.asList(Arrays.asList(new Tuple2<>(1.0, 'x'))); JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream( JavaTestUtils.attachTestInputStream(ssc, pairInputData1, 1)); JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream( @@ -293,13 +294,13 @@ public void testStreamingContextTransform() { ); List>> pairStream1input = Arrays.asList( - Arrays.asList(new Tuple2(1, "x")), - Arrays.asList(new Tuple2(2, "y")) + Arrays.asList(new Tuple2<>(1, "x")), + Arrays.asList(new Tuple2<>(2, "y")) ); List>>> expected = Arrays.asList( - Arrays.asList(new Tuple2>(1, new Tuple2(1, "x"))), - Arrays.asList(new Tuple2>(2, new Tuple2(2, "y"))) + Arrays.asList(new Tuple2<>(1, new Tuple2<>(1, "x"))), + Arrays.asList(new Tuple2<>(2, new Tuple2<>(2, "y"))) ); JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, stream1input, 1); @@ -312,7 +313,7 @@ public void testStreamingContextTransform() { // This is just to test whether this transform to JavaStream compiles JavaDStream transformed1 = ssc.transform( listOfDStreams1, (List> listOfRDDs, Time time) -> { - assert (listOfRDDs.size() == 2); + Assert.assertEquals(2, listOfRDDs.size()); return null; }); @@ -321,13 +322,13 @@ public void testStreamingContextTransform() { JavaPairDStream> transformed2 = ssc.transformToPair( listOfDStreams2, (List> listOfRDDs, Time time) -> { - assert (listOfRDDs.size() == 3); + Assert.assertEquals(3, listOfRDDs.size()); JavaRDD rdd1 = (JavaRDD) listOfRDDs.get(0); JavaRDD rdd2 = (JavaRDD) listOfRDDs.get(1); JavaRDD> rdd3 = (JavaRDD>) listOfRDDs.get(2); JavaPairRDD prdd3 = JavaPairRDD.fromJavaRDD(rdd3); PairFunction mapToTuple = - (Integer i) -> new Tuple2(i, i); + (Integer i) -> new Tuple2<>(i, i); return rdd1.union(rdd2).mapToPair(mapToTuple).join(prdd3); }); JavaTestUtils.attachTestOutputStream(transformed2); @@ -365,36 +366,36 @@ public void testPairFlatMap() { List>> expected = Arrays.asList( Arrays.asList( - new Tuple2(6, "g"), - new Tuple2(6, "i"), - new Tuple2(6, "a"), - new Tuple2(6, "n"), - new Tuple2(6, "t"), - new Tuple2(6, "s")), + new Tuple2<>(6, "g"), + new Tuple2<>(6, "i"), + new Tuple2<>(6, "a"), + new Tuple2<>(6, "n"), + new Tuple2<>(6, "t"), + new Tuple2<>(6, "s")), Arrays.asList( - new Tuple2(7, "d"), - new Tuple2(7, "o"), - new Tuple2(7, "d"), - new Tuple2(7, "g"), - new Tuple2(7, "e"), - new Tuple2(7, "r"), - new Tuple2(7, "s")), + new Tuple2<>(7, "d"), + new Tuple2<>(7, "o"), + new Tuple2<>(7, "d"), + new Tuple2<>(7, "g"), + new Tuple2<>(7, "e"), + new Tuple2<>(7, "r"), + new Tuple2<>(7, "s")), Arrays.asList( - new Tuple2(9, "a"), - new Tuple2(9, "t"), - new Tuple2(9, "h"), - new Tuple2(9, "l"), - new Tuple2(9, "e"), - new Tuple2(9, "t"), - new Tuple2(9, "i"), - new Tuple2(9, "c"), - new Tuple2(9, "s"))); + new Tuple2<>(9, "a"), + new Tuple2<>(9, "t"), + new Tuple2<>(9, "h"), + new Tuple2<>(9, "l"), + new Tuple2<>(9, "e"), + new Tuple2<>(9, "t"), + new Tuple2<>(9, "i"), + new Tuple2<>(9, "c"), + new Tuple2<>(9, "s"))); JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream flatMapped = stream.flatMapToPair(s -> { List> out = Lists.newArrayList(); for (String letter : s.split("(?!^)")) { - out.add(new Tuple2(s.length(), letter)); + out.add(new Tuple2<>(s.length(), letter)); } return out; }); @@ -411,12 +412,8 @@ public void testPairFlatMap() { */ public static > void assertOrderInvariantEquals( List> expected, List> actual) { - for (List list : expected) { - Collections.sort(list); - } - for (List list : actual) { - Collections.sort(list); - } + expected.forEach((List list) -> Collections.sort(list)); + actual.forEach((List list) -> Collections.sort(list)); Assert.assertEquals(expected, actual); } @@ -424,11 +421,11 @@ public static > void assertOrderInvariantEquals( public void testPairFilter() { List> inputData = Arrays.asList( Arrays.asList("giants", "dodgers"), - Arrays.asList("yankees", "red socks")); + Arrays.asList("yankees", "red sox")); List>> expected = Arrays.asList( - Arrays.asList(new Tuple2("giants", 6)), - Arrays.asList(new Tuple2("yankees", 7))); + Arrays.asList(new Tuple2<>("giants", 6)), + Arrays.asList(new Tuple2<>("yankees", 7))); JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = @@ -441,26 +438,26 @@ public void testPairFilter() { } List>> stringStringKVStream = Arrays.asList( - Arrays.asList(new Tuple2("california", "dodgers"), - new Tuple2("california", "giants"), - new Tuple2("new york", "yankees"), - new Tuple2("new york", "mets")), - Arrays.asList(new Tuple2("california", "sharks"), - new Tuple2("california", "ducks"), - new Tuple2("new york", "rangers"), - new Tuple2("new york", "islanders"))); + Arrays.asList(new Tuple2<>("california", "dodgers"), + new Tuple2<>("california", "giants"), + new Tuple2<>("new york", "yankees"), + new Tuple2<>("new york", "mets")), + Arrays.asList(new Tuple2<>("california", "sharks"), + new Tuple2<>("california", "ducks"), + new Tuple2<>("new york", "rangers"), + new Tuple2<>("new york", "islanders"))); List>> stringIntKVStream = Arrays.asList( Arrays.asList( - new Tuple2("california", 1), - new Tuple2("california", 3), - new Tuple2("new york", 4), - new Tuple2("new york", 1)), + new Tuple2<>("california", 1), + new Tuple2<>("california", 3), + new Tuple2<>("new york", 4), + new Tuple2<>("new york", 1)), Arrays.asList( - new Tuple2("california", 5), - new Tuple2("california", 5), - new Tuple2("new york", 3), - new Tuple2("new york", 1))); + new Tuple2<>("california", 5), + new Tuple2<>("california", 5), + new Tuple2<>("new york", 3), + new Tuple2<>("new york", 1))); @Test public void testPairMap() { // Maps pair -> pair of different type @@ -468,15 +465,15 @@ public void testPairMap() { // Maps pair -> pair of different type List>> expected = Arrays.asList( Arrays.asList( - new Tuple2(1, "california"), - new Tuple2(3, "california"), - new Tuple2(4, "new york"), - new Tuple2(1, "new york")), + new Tuple2<>(1, "california"), + new Tuple2<>(3, "california"), + new Tuple2<>(4, "new york"), + new Tuple2<>(1, "new york")), Arrays.asList( - new Tuple2(5, "california"), - new Tuple2(5, "california"), - new Tuple2(3, "new york"), - new Tuple2(1, "new york"))); + new Tuple2<>(5, "california"), + new Tuple2<>(5, "california"), + new Tuple2<>(3, "new york"), + new Tuple2<>(1, "new york"))); JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); @@ -494,21 +491,21 @@ public void testPairMapPartitions() { // Maps pair -> pair of different type List>> expected = Arrays.asList( Arrays.asList( - new Tuple2(1, "california"), - new Tuple2(3, "california"), - new Tuple2(4, "new york"), - new Tuple2(1, "new york")), + new Tuple2<>(1, "california"), + new Tuple2<>(3, "california"), + new Tuple2<>(4, "new york"), + new Tuple2<>(1, "new york")), Arrays.asList( - new Tuple2(5, "california"), - new Tuple2(5, "california"), - new Tuple2(3, "new york"), - new Tuple2(1, "new york"))); + new Tuple2<>(5, "california"), + new Tuple2<>(5, "california"), + new Tuple2<>(3, "new york"), + new Tuple2<>(1, "new york"))); JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); JavaPairDStream reversed = pairStream.mapPartitionsToPair(in -> { - LinkedList> out = new LinkedList>(); + LinkedList> out = new LinkedList<>(); while (in.hasNext()) { Tuple2 next = in.next(); out.add(next.swap()); @@ -530,7 +527,8 @@ public void testPairMap2() { // Maps pair -> single Arrays.asList(1, 3, 4, 1), Arrays.asList(5, 5, 3, 1)); - JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); JavaDStream reversed = pairStream.map(in -> in._2()); JavaTestUtils.attachTestOutputStream(reversed); @@ -543,31 +541,31 @@ public void testPairMap2() { // Maps pair -> single public void testPairToPairFlatMapWithChangingTypes() { // Maps pair -> pair List>> inputData = Arrays.asList( Arrays.asList( - new Tuple2("hi", 1), - new Tuple2("ho", 2)), + new Tuple2<>("hi", 1), + new Tuple2<>("ho", 2)), Arrays.asList( - new Tuple2("hi", 1), - new Tuple2("ho", 2))); + new Tuple2<>("hi", 1), + new Tuple2<>("ho", 2))); List>> expected = Arrays.asList( Arrays.asList( - new Tuple2(1, "h"), - new Tuple2(1, "i"), - new Tuple2(2, "h"), - new Tuple2(2, "o")), + new Tuple2<>(1, "h"), + new Tuple2<>(1, "i"), + new Tuple2<>(2, "h"), + new Tuple2<>(2, "o")), Arrays.asList( - new Tuple2(1, "h"), - new Tuple2(1, "i"), - new Tuple2(2, "h"), - new Tuple2(2, "o"))); + new Tuple2<>(1, "h"), + new Tuple2<>(1, "i"), + new Tuple2<>(2, "h"), + new Tuple2<>(2, "o"))); JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); JavaPairDStream flatMapped = pairStream.flatMapToPair(in -> { - List> out = new LinkedList>(); + List> out = new LinkedList<>(); for (Character s : in._1().toCharArray()) { - out.add(new Tuple2(in._2(), s.toString())); + out.add(new Tuple2<>(in._2(), s.toString())); } return out; }); @@ -584,11 +582,11 @@ public void testPairReduceByKey() { List>> expected = Arrays.asList( Arrays.asList( - new Tuple2("california", 4), - new Tuple2("new york", 5)), + new Tuple2<>("california", 4), + new Tuple2<>("new york", 5)), Arrays.asList( - new Tuple2("california", 10), - new Tuple2("new york", 4))); + new Tuple2<>("california", 10), + new Tuple2<>("new york", 4))); JavaDStream> stream = JavaTestUtils.attachTestInputStream( ssc, inputData, 1); @@ -608,11 +606,11 @@ public void testCombineByKey() { List>> expected = Arrays.asList( Arrays.asList( - new Tuple2("california", 4), - new Tuple2("new york", 5)), + new Tuple2<>("california", 4), + new Tuple2<>("new york", 5)), Arrays.asList( - new Tuple2("california", 10), - new Tuple2("new york", 4))); + new Tuple2<>("california", 10), + new Tuple2<>("new york", 4))); JavaDStream> stream = JavaTestUtils.attachTestInputStream( ssc, inputData, 1); @@ -632,12 +630,12 @@ public void testReduceByKeyAndWindow() { List>> inputData = stringIntKVStream; List>> expected = Arrays.asList( - Arrays.asList(new Tuple2("california", 4), - new Tuple2("new york", 5)), - Arrays.asList(new Tuple2("california", 14), - new Tuple2("new york", 9)), - Arrays.asList(new Tuple2("california", 10), - new Tuple2("new york", 4))); + Arrays.asList(new Tuple2<>("california", 4), + new Tuple2<>("new york", 5)), + Arrays.asList(new Tuple2<>("california", 14), + new Tuple2<>("new york", 9)), + Arrays.asList(new Tuple2<>("california", 10), + new Tuple2<>("new york", 4))); JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); @@ -656,12 +654,12 @@ public void testUpdateStateByKey() { List>> inputData = stringIntKVStream; List>> expected = Arrays.asList( - Arrays.asList(new Tuple2("california", 4), - new Tuple2("new york", 5)), - Arrays.asList(new Tuple2("california", 14), - new Tuple2("new york", 9)), - Arrays.asList(new Tuple2("california", 14), - new Tuple2("new york", 9))); + Arrays.asList(new Tuple2<>("california", 4), + new Tuple2<>("new york", 5)), + Arrays.asList(new Tuple2<>("california", 14), + new Tuple2<>("new york", 9)), + Arrays.asList(new Tuple2<>("california", 14), + new Tuple2<>("new york", 9))); JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); @@ -689,12 +687,12 @@ public void testReduceByKeyAndWindowWithInverse() { List>> inputData = stringIntKVStream; List>> expected = Arrays.asList( - Arrays.asList(new Tuple2("california", 4), - new Tuple2("new york", 5)), - Arrays.asList(new Tuple2("california", 14), - new Tuple2("new york", 9)), - Arrays.asList(new Tuple2("california", 10), - new Tuple2("new york", 4))); + Arrays.asList(new Tuple2<>("california", 4), + new Tuple2<>("new york", 5)), + Arrays.asList(new Tuple2<>("california", 14), + new Tuple2<>("new york", 9)), + Arrays.asList(new Tuple2<>("california", 10), + new Tuple2<>("new york", 4))); JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); @@ -713,27 +711,27 @@ public void testReduceByKeyAndWindowWithInverse() { public void testPairTransform() { List>> inputData = Arrays.asList( Arrays.asList( - new Tuple2(3, 5), - new Tuple2(1, 5), - new Tuple2(4, 5), - new Tuple2(2, 5)), + new Tuple2<>(3, 5), + new Tuple2<>(1, 5), + new Tuple2<>(4, 5), + new Tuple2<>(2, 5)), Arrays.asList( - new Tuple2(2, 5), - new Tuple2(3, 5), - new Tuple2(4, 5), - new Tuple2(1, 5))); + new Tuple2<>(2, 5), + new Tuple2<>(3, 5), + new Tuple2<>(4, 5), + new Tuple2<>(1, 5))); List>> expected = Arrays.asList( Arrays.asList( - new Tuple2(1, 5), - new Tuple2(2, 5), - new Tuple2(3, 5), - new Tuple2(4, 5)), + new Tuple2<>(1, 5), + new Tuple2<>(2, 5), + new Tuple2<>(3, 5), + new Tuple2<>(4, 5)), Arrays.asList( - new Tuple2(1, 5), - new Tuple2(2, 5), - new Tuple2(3, 5), - new Tuple2(4, 5))); + new Tuple2<>(1, 5), + new Tuple2<>(2, 5), + new Tuple2<>(3, 5), + new Tuple2<>(4, 5))); JavaDStream> stream = JavaTestUtils.attachTestInputStream( ssc, inputData, 1); @@ -751,15 +749,15 @@ public void testPairTransform() { public void testPairToNormalRDDTransform() { List>> inputData = Arrays.asList( Arrays.asList( - new Tuple2(3, 5), - new Tuple2(1, 5), - new Tuple2(4, 5), - new Tuple2(2, 5)), + new Tuple2<>(3, 5), + new Tuple2<>(1, 5), + new Tuple2<>(4, 5), + new Tuple2<>(2, 5)), Arrays.asList( - new Tuple2(2, 5), - new Tuple2(3, 5), - new Tuple2(4, 5), - new Tuple2(1, 5))); + new Tuple2<>(2, 5), + new Tuple2<>(3, 5), + new Tuple2<>(4, 5), + new Tuple2<>(1, 5))); List> expected = Arrays.asList( Arrays.asList(3, 1, 4, 2), @@ -780,20 +778,20 @@ public void testMapValues() { List>> inputData = stringStringKVStream; List>> expected = Arrays.asList( - Arrays.asList(new Tuple2("california", "DODGERS"), - new Tuple2("california", "GIANTS"), - new Tuple2("new york", "YANKEES"), - new Tuple2("new york", "METS")), - Arrays.asList(new Tuple2("california", "SHARKS"), - new Tuple2("california", "DUCKS"), - new Tuple2("new york", "RANGERS"), - new Tuple2("new york", "ISLANDERS"))); + Arrays.asList(new Tuple2<>("california", "DODGERS"), + new Tuple2<>("california", "GIANTS"), + new Tuple2<>("new york", "YANKEES"), + new Tuple2<>("new york", "METS")), + Arrays.asList(new Tuple2<>("california", "SHARKS"), + new Tuple2<>("california", "DUCKS"), + new Tuple2<>("new york", "RANGERS"), + new Tuple2<>("new york", "ISLANDERS"))); JavaDStream> stream = JavaTestUtils.attachTestInputStream( ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream mapped = pairStream.mapValues(s -> s.toUpperCase()); + JavaPairDStream mapped = pairStream.mapValues(String::toUpperCase); JavaTestUtils.attachTestOutputStream(mapped); List>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -805,34 +803,29 @@ public void testFlatMapValues() { List>> inputData = stringStringKVStream; List>> expected = Arrays.asList( - Arrays.asList(new Tuple2("california", "dodgers1"), - new Tuple2("california", "dodgers2"), - new Tuple2("california", "giants1"), - new Tuple2("california", "giants2"), - new Tuple2("new york", "yankees1"), - new Tuple2("new york", "yankees2"), - new Tuple2("new york", "mets1"), - new Tuple2("new york", "mets2")), - Arrays.asList(new Tuple2("california", "sharks1"), - new Tuple2("california", "sharks2"), - new Tuple2("california", "ducks1"), - new Tuple2("california", "ducks2"), - new Tuple2("new york", "rangers1"), - new Tuple2("new york", "rangers2"), - new Tuple2("new york", "islanders1"), - new Tuple2("new york", "islanders2"))); + Arrays.asList(new Tuple2<>("california", "dodgers1"), + new Tuple2<>("california", "dodgers2"), + new Tuple2<>("california", "giants1"), + new Tuple2<>("california", "giants2"), + new Tuple2<>("new york", "yankees1"), + new Tuple2<>("new york", "yankees2"), + new Tuple2<>("new york", "mets1"), + new Tuple2<>("new york", "mets2")), + Arrays.asList(new Tuple2<>("california", "sharks1"), + new Tuple2<>("california", "sharks2"), + new Tuple2<>("california", "ducks1"), + new Tuple2<>("california", "ducks2"), + new Tuple2<>("new york", "rangers1"), + new Tuple2<>("new york", "rangers2"), + new Tuple2<>("new york", "islanders1"), + new Tuple2<>("new york", "islanders2"))); JavaDStream> stream = JavaTestUtils.attachTestInputStream( ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream flatMapped = pairStream.flatMapValues(in -> { - List out = new ArrayList(); - out.add(in + "1"); - out.add(in + "2"); - return out; - }); + JavaPairDStream flatMapped = + pairStream.flatMapValues(in -> Arrays.asList(in + "1", in + "2")); JavaTestUtils.attachTestOutputStream(flatMapped); List>> result = JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected, result); From 189df165bb7cb8bc8ede48d0e7f8d8b5cd31d299 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 4 Jun 2014 12:56:56 -0700 Subject: [PATCH 22/54] [SPARK-1752][MLLIB] Standardize text format for vectors and labeled points We should standardize the text format used to represent vectors and labeled points. The proposed formats are the following: 1. dense vector: `[v0,v1,..]` 2. sparse vector: `(size,[i0,i1],[v0,v1])` 3. labeled point: `(label,vector)` where "(..)" indicates a tuple and "[...]" indicate an array. `loadLabeledPoints` is added to pyspark's `MLUtils`. I didn't add `loadVectors` to pyspark because `RDD.saveAsTextFile` cannot stringify dense vectors in the proposed format automatically. `MLUtils#saveLabeledData` and `MLUtils#loadLabeledData` are deprecated. Users should use `RDD#saveAsTextFile` and `MLUtils#loadLabeledPoints` instead. In Scala, `MLUtils#loadLabeledPoints` is compatible with the format used by `MLUtils#loadLabeledData`. CC: @mateiz, @srowen Author: Xiangrui Meng Closes #685 from mengxr/labeled-io and squashes the following commits: 2d1116a [Xiangrui Meng] make loadLabeledData/saveLabeledData deprecated since 1.0.1 297be75 [Xiangrui Meng] change LabeledPoint.parse to LabeledPointParser.parse to maintain binary compatibility d6b1473 [Xiangrui Meng] Merge branch 'master' into labeled-io 56746ea [Xiangrui Meng] replace # by . 623a5f0 [Xiangrui Meng] merge master f06d5ba [Xiangrui Meng] add docs and minor updates 640fe0c [Xiangrui Meng] throw SparkException 5bcfbc4 [Xiangrui Meng] update test to add scientific notations e86bf38 [Xiangrui Meng] remove NumericTokenizer 050fca4 [Xiangrui Meng] use StringTokenizer 6155b75 [Xiangrui Meng] merge master f644438 [Xiangrui Meng] remove parse methods based on eval from pyspark a41675a [Xiangrui Meng] python loadLabeledPoint uses Scala's implementation ce9a475 [Xiangrui Meng] add deserialize_labeled_point to pyspark with tests e9fcd49 [Xiangrui Meng] add serializeLabeledPoint and tests aea4ae3 [Xiangrui Meng] minor updates 810d6df [Xiangrui Meng] update tokenizer/parser implementation 7aac03a [Xiangrui Meng] remove Scala parsers c1885c1 [Xiangrui Meng] add headers and minor changes b0c50cb [Xiangrui Meng] add customized parser d731817 [Xiangrui Meng] style update 63dc396 [Xiangrui Meng] add loadLabeledPoints to pyspark ea122b5 [Xiangrui Meng] Merge branch 'master' into labeled-io cd6c78f [Xiangrui Meng] add __str__ and parse to LabeledPoint a7a178e [Xiangrui Meng] add stringify to pyspark's Vectors 5c2dbfa [Xiangrui Meng] add parse to pyspark's Vectors 7853f88 [Xiangrui Meng] update pyspark's SparseVector.__str__ e761d32 [Xiangrui Meng] make LabelPoint.parse compatible with the dense format used before v1.0 and deprecate loadLabeledData and saveLabeledData 9e63a02 [Xiangrui Meng] add loadVectors and loadLabeledPoints 19aa523 [Xiangrui Meng] update toString and add parsers for Vectors and LabeledPoint --- .../examples/mllib/DecisionTreeRunner.scala | 2 +- .../mllib/api/python/PythonMLLibAPI.scala | 33 ++++- .../apache/spark/mllib/linalg/Vectors.scala | 33 ++++- .../spark/mllib/regression/LabeledPoint.scala | 31 ++++- .../mllib/util/LinearDataGenerator.scala | 3 +- .../LogisticRegressionDataGenerator.scala | 3 +- .../org/apache/spark/mllib/util/MLUtils.scala | 47 ++++++- .../spark/mllib/util/NumericParser.scala | 121 ++++++++++++++++++ .../spark/mllib/util/SVMDataGenerator.scala | 2 +- .../api/python/PythonMLLibAPISuite.scala | 60 +++++++++ .../spark/mllib/linalg/VectorsSuite.scala | 25 ++++ .../mllib/regression/LabeledPointSuite.scala | 39 ++++++ .../spark/mllib/util/MLUtilsSuite.scala | 30 ++++- .../spark/mllib/util/NumericParserSuite.scala | 42 ++++++ python/pyspark/mllib/_common.py | 72 ++++++++--- python/pyspark/mllib/linalg.py | 34 +++-- python/pyspark/mllib/regression.py | 5 +- python/pyspark/mllib/util.py | 69 +++++++--- 18 files changed, 579 insertions(+), 72 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/NumericParser.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/util/NumericParserSuite.scala 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 9832bec90d7ee..b3cc361154198 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 @@ -99,7 +99,7 @@ object DecisionTreeRunner { val sc = new SparkContext(conf) // Load training data and cache it. - val examples = MLUtils.loadLabeledData(sc, params.input).cache() + val examples = MLUtils.loadLabeledPoints(sc, params.input).cache() val splits = examples.randomSplit(Array(0.8, 0.2)) val training = splits(0).cache() 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 7c65b0d4750fa..c44173793b39a 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 @@ -20,12 +20,13 @@ 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.api.java.{JavaSparkContext, JavaRDD} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} import org.apache.spark.mllib.recommendation._ import org.apache.spark.mllib.regression._ +import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD /** @@ -41,7 +42,7 @@ class PythonMLLibAPI extends Serializable { private val DENSE_MATRIX_MAGIC: Byte = 3 private val LABELED_POINT_MAGIC: Byte = 4 - private def deserializeDoubleVector(bytes: Array[Byte], offset: Int = 0): Vector = { + private[python] def deserializeDoubleVector(bytes: Array[Byte], offset: Int = 0): Vector = { require(bytes.length - offset >= 5, "Byte array too short") val magic = bytes(offset) if (magic == DENSE_VECTOR_MAGIC) { @@ -116,7 +117,7 @@ class PythonMLLibAPI extends Serializable { bytes } - private def serializeDoubleVector(vector: Vector): Array[Byte] = vector match { + private[python] def serializeDoubleVector(vector: Vector): Array[Byte] = vector match { case s: SparseVector => serializeSparseVector(s) case _ => @@ -167,7 +168,18 @@ class PythonMLLibAPI extends Serializable { bytes } - private def deserializeLabeledPoint(bytes: Array[Byte]): LabeledPoint = { + private[python] def serializeLabeledPoint(p: LabeledPoint): Array[Byte] = { + val fb = serializeDoubleVector(p.features) + val bytes = new Array[Byte](1 + 8 + fb.length) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + bb.put(LABELED_POINT_MAGIC) + bb.putDouble(p.label) + bb.put(fb) + bytes + } + + private[python] def deserializeLabeledPoint(bytes: Array[Byte]): LabeledPoint = { require(bytes.length >= 9, "Byte array too short") val magic = bytes(0) if (magic != LABELED_POINT_MAGIC) { @@ -179,6 +191,19 @@ class PythonMLLibAPI extends Serializable { LabeledPoint(label, deserializeDoubleVector(bytes, 9)) } + /** + * Loads and serializes labeled points saved with `RDD#saveAsTextFile`. + * @param jsc Java SparkContext + * @param path file or directory path in any Hadoop-supported file system URI + * @param minPartitions min number of partitions + * @return serialized labeled points stored in a JavaRDD of byte array + */ + def loadLabeledPoints( + jsc: JavaSparkContext, + path: String, + minPartitions: Int): JavaRDD[Array[Byte]] = + MLUtils.loadLabeledPoints(jsc.sc, path, minPartitions).map(serializeLabeledPoint).toJavaRDD() + private def trainRegressionModel( trainFunc: (RDD[LabeledPoint], Vector) => GeneralizedLinearModel, dataBytesJRDD: JavaRDD[Array[Byte]], 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 84d223908c1f6..c818a0b9c3e43 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 @@ -17,13 +17,16 @@ package org.apache.spark.mllib.linalg -import java.lang.{Iterable => JavaIterable, Integer => JavaInteger, Double => JavaDouble} +import java.lang.{Double => JavaDouble, Integer => JavaInteger, Iterable => JavaIterable} import java.util.Arrays import scala.annotation.varargs import scala.collection.JavaConverters._ -import breeze.linalg.{Vector => BV, DenseVector => BDV, SparseVector => BSV} +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} + +import org.apache.spark.mllib.util.NumericParser +import org.apache.spark.SparkException /** * Represents a numeric vector, whose index type is Int and value type is Double. @@ -124,6 +127,25 @@ object Vectors { }.toSeq) } + /** + * Parses a string resulted from `Vector#toString` into + * an [[org.apache.spark.mllib.linalg.Vector]]. + */ + def parse(s: String): Vector = { + parseNumeric(NumericParser.parse(s)) + } + + private[mllib] def parseNumeric(any: Any): Vector = { + any match { + case values: Array[Double] => + Vectors.dense(values) + case Seq(size: Double, indices: Array[Double], values: Array[Double]) => + Vectors.sparse(size.toInt, indices.map(_.toInt), values) + case other => + throw new SparkException(s"Cannot parse $other.") + } + } + /** * Creates a vector instance from a breeze vector. */ @@ -175,9 +197,10 @@ class SparseVector( val indices: Array[Int], val values: Array[Double]) extends Vector { - override def toString: String = { - "(" + size + "," + indices.zip(values).mkString("[", "," ,"]") + ")" - } + require(indices.length == values.length) + + override def toString: String = + "(%s,%s,%s)".format(size, indices.mkString("[", ",", "]"), values.mkString("[", ",", "]")) override def toArray: Array[Double] = { val data = new Array[Double](size) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala index 3deab1ab785b9..62a03af4a9964 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala @@ -17,7 +17,9 @@ package org.apache.spark.mllib.regression -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.mllib.util.NumericParser +import org.apache.spark.SparkException /** * Class that represents the features and labels of a data point. @@ -27,6 +29,31 @@ import org.apache.spark.mllib.linalg.Vector */ case class LabeledPoint(label: Double, features: Vector) { override def toString: String = { - "LabeledPoint(%s, %s)".format(label, features) + "(%s,%s)".format(label, features) + } +} + +/** + * Parser for [[org.apache.spark.mllib.regression.LabeledPoint]]. + */ +private[mllib] object LabeledPointParser { + /** + * Parses a string resulted from `LabeledPoint#toString` into + * an [[org.apache.spark.mllib.regression.LabeledPoint]]. + */ + def parse(s: String): LabeledPoint = { + if (s.startsWith("(")) { + NumericParser.parse(s) match { + case Seq(label: Double, numeric: Any) => + LabeledPoint(label, Vectors.parseNumeric(numeric)) + case other => + throw new SparkException(s"Cannot parse $other.") + } + } else { // dense format used before v1.0 + val parts = s.split(',') + val label = java.lang.Double.parseDouble(parts(0)) + val features = Vectors.dense(parts(1).trim().split(' ').map(java.lang.Double.parseDouble)) + LabeledPoint(label, features) + } } } 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 c8e160d00c2d6..69299c219878c 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 @@ -129,7 +129,8 @@ object LinearDataGenerator { val sc = new SparkContext(sparkMaster, "LinearDataGenerator") val data = generateLinearRDD(sc, nexamples, nfeatures, eps, nparts = parts) - MLUtils.saveLabeledData(data, outputPath) + data.saveAsTextFile(outputPath) + sc.stop() } } 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 c82cd8fd4641c..9d802678c4a77 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 @@ -79,7 +79,8 @@ object LogisticRegressionDataGenerator { val sc = new SparkContext(sparkMaster, "LogisticRegressionDataGenerator") val data = generateLogisticRDD(sc, nexamples, nfeatures, eps, parts) - MLUtils.saveLabeledData(data, outputPath) + data.saveAsTextFile(outputPath) + sc.stop() } } 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 e598b6cb171a8..aaf92a1a8869a 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 @@ -27,7 +27,7 @@ import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.rdd.PartitionwiseSampledRDD import org.apache.spark.util.random.BernoulliSampler -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPointParser, LabeledPoint} import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.storage.StorageLevel @@ -180,7 +180,39 @@ object MLUtils { } /** - * :: Experimental :: + * Loads vectors saved using `RDD[Vector].saveAsTextFile`. + * @param sc Spark context + * @param path file or directory path in any Hadoop-supported file system URI + * @param minPartitions min number of partitions + * @return vectors stored as an RDD[Vector] + */ + def loadVectors(sc: SparkContext, path: String, minPartitions: Int): RDD[Vector] = + sc.textFile(path, minPartitions).map(Vectors.parse) + + /** + * Loads vectors saved using `RDD[Vector].saveAsTextFile` with the default number of partitions. + */ + def loadVectors(sc: SparkContext, path: String): RDD[Vector] = + sc.textFile(path, sc.defaultMinPartitions).map(Vectors.parse) + + /** + * Loads labeled points saved using `RDD[LabeledPoint].saveAsTextFile`. + * @param sc Spark context + * @param path file or directory path in any Hadoop-supported file system URI + * @param minPartitions min number of partitions + * @return labeled points stored as an RDD[LabeledPoint] + */ + def loadLabeledPoints(sc: SparkContext, path: String, minPartitions: Int): RDD[LabeledPoint] = + sc.textFile(path, minPartitions).map(LabeledPointParser.parse) + + /** + * Loads labeled points saved using `RDD[LabeledPoint].saveAsTextFile` with the default number of + * partitions. + */ + def loadLabeledPoints(sc: SparkContext, dir: String): RDD[LabeledPoint] = + loadLabeledPoints(sc, dir, sc.defaultMinPartitions) + + /** * 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. @@ -189,8 +221,11 @@ object MLUtils { * @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). + * + * @deprecated Should use [[org.apache.spark.rdd.RDD#saveAsTextFile]] for saving and + * [[org.apache.spark.mllib.util.MLUtils#loadLabeledPoints]] for loading. */ - @Experimental + @deprecated("Should use MLUtils.loadLabeledPoints instead.", "1.0.1") def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { sc.textFile(dir).map { line => val parts = line.split(',') @@ -201,15 +236,17 @@ 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. * * @param data An RDD of LabeledPoints containing data to be saved. * @param dir Directory to save the data. + * + * @deprecated Should use [[org.apache.spark.rdd.RDD#saveAsTextFile]] for saving and + * [[org.apache.spark.mllib.util.MLUtils#loadLabeledPoints]] for loading. */ - @Experimental + @deprecated("Should use RDD[LabeledPoint].saveAsTextFile instead.", "1.0.1") def saveLabeledData(data: RDD[LabeledPoint], dir: String) { val dataStr = data.map(x => x.label + "," + x.features.toArray.mkString(" ")) dataStr.saveAsTextFile(dir) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/NumericParser.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/NumericParser.scala new file mode 100644 index 0000000000000..f7cba6c6cb628 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/NumericParser.scala @@ -0,0 +1,121 @@ +/* + * 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 java.util.StringTokenizer + +import scala.collection.mutable.{ArrayBuffer, ListBuffer} + +import org.apache.spark.SparkException + +/** + * Simple parser for a numeric structure consisting of three types: + * + * - number: a double in Java's floating number format + * - array: an array of numbers stored as `[v0,v1,...,vn]` + * - tuple: a list of numbers, arrays, or tuples stored as `(...)` + */ +private[mllib] object NumericParser { + + /** Parses a string into a Double, an Array[Double], or a Seq[Any]. */ + def parse(s: String): Any = { + val tokenizer = new StringTokenizer(s, "()[],", true) + if (tokenizer.hasMoreTokens()) { + val token = tokenizer.nextToken() + if (token == "(") { + parseTuple(tokenizer) + } else if (token == "[") { + parseArray(tokenizer) + } else { + // expecting a number + parseDouble(token) + } + } else { + throw new SparkException(s"Cannot find any token from the input string.") + } + } + + private def parseArray(tokenizer: StringTokenizer): Array[Double] = { + val values = ArrayBuffer.empty[Double] + var parsing = true + var allowComma = false + var token: String = null + while (parsing && tokenizer.hasMoreTokens()) { + token = tokenizer.nextToken() + if (token == "]") { + parsing = false + } else if (token == ",") { + if (allowComma) { + allowComma = false + } else { + throw new SparkException("Found a ',' at a wrong position.") + } + } else { + // expecting a number + values.append(parseDouble(token)) + allowComma = true + } + } + if (parsing) { + throw new SparkException(s"An array must end with ']'.") + } + values.toArray + } + + private def parseTuple(tokenizer: StringTokenizer): Seq[_] = { + val items = ListBuffer.empty[Any] + var parsing = true + var allowComma = false + var token: String = null + while (parsing && tokenizer.hasMoreTokens()) { + token = tokenizer.nextToken() + if (token == "(") { + items.append(parseTuple(tokenizer)) + allowComma = true + } else if (token == "[") { + items.append(parseArray(tokenizer)) + allowComma = true + } else if (token == ",") { + if (allowComma) { + allowComma = false + } else { + throw new SparkException("Found a ',' at a wrong position.") + } + } else if (token == ")") { + parsing = false + } else { + // expecting a number + items.append(parseDouble(token)) + allowComma = true + } + } + if (parsing) { + throw new SparkException(s"A tuple must end with ')'.") + } + items + } + + private def parseDouble(s: String): Double = { + try { + java.lang.Double.parseDouble(s) + } catch { + case e: Throwable => + throw new SparkException(s"Cannot parse a double from: $s", e) + } + } +} 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 ba8190b0e07e8..7db97e6bac688 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 @@ -65,7 +65,7 @@ object SVMDataGenerator { LabeledPoint(y, Vectors.dense(x)) } - MLUtils.saveLabeledData(data, outputPath) + data.saveAsTextFile(outputPath) sc.stop() } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala new file mode 100644 index 0000000000000..642843f90204c --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala @@ -0,0 +1,60 @@ +/* + * 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.python + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint + +class PythonMLLibAPISuite extends FunSuite { + val py = new PythonMLLibAPI + + test("vector serialization") { + val vectors = Seq( + Vectors.dense(Array.empty[Double]), + Vectors.dense(0.0), + Vectors.dense(0.0, -2.0), + Vectors.sparse(0, Array.empty[Int], Array.empty[Double]), + Vectors.sparse(1, Array.empty[Int], Array.empty[Double]), + Vectors.sparse(2, Array(1), Array(-2.0))) + vectors.foreach { v => + val bytes = py.serializeDoubleVector(v) + val u = py.deserializeDoubleVector(bytes) + assert(u.getClass === v.getClass) + assert(u === v) + } + } + + test("labeled point serialization") { + val points = Seq( + LabeledPoint(0.0, Vectors.dense(Array.empty[Double])), + LabeledPoint(1.0, Vectors.dense(0.0)), + LabeledPoint(-0.5, Vectors.dense(0.0, -2.0)), + LabeledPoint(0.0, Vectors.sparse(0, Array.empty[Int], Array.empty[Double])), + LabeledPoint(1.0, Vectors.sparse(1, Array.empty[Int], Array.empty[Double])), + LabeledPoint(-0.5, Vectors.sparse(2, Array(1), Array(-2.0)))) + points.foreach { p => + val bytes = py.serializeLabeledPoint(p) + val q = py.deserializeLabeledPoint(bytes) + assert(q.label === p.label) + assert(q.features.getClass === p.features.getClass) + assert(q.features === p.features) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index cfe8a27fcb71e..7972ceea1fe8a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.mllib.linalg import org.scalatest.FunSuite +import org.apache.spark.SparkException + class VectorsSuite extends FunSuite { val arr = Array(0.1, 0.0, 0.3, 0.4) @@ -100,4 +102,27 @@ class VectorsSuite extends FunSuite { assert(vec2(6) === 4.0) assert(vec2(7) === 0.0) } + + test("parse vectors") { + val vectors = Seq( + Vectors.dense(Array.empty[Double]), + Vectors.dense(1.0), + Vectors.dense(1.0E6, 0.0, -2.0e-7), + Vectors.sparse(0, Array.empty[Int], Array.empty[Double]), + Vectors.sparse(1, Array(0), Array(1.0)), + Vectors.sparse(3, Array(0, 2), Array(1.0, -2.0))) + vectors.foreach { v => + val v1 = Vectors.parse(v.toString) + assert(v.getClass === v1.getClass) + assert(v === v1) + } + + val malformatted = Seq("1", "[1,,]", "[1,2b]", "(1,[1,2])", "([1],[2.0,1.0])") + malformatted.foreach { s => + intercept[SparkException] { + Vectors.parse(s) + println(s"Didn't detect malformatted string $s.") + } + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala new file mode 100644 index 0000000000000..d9308aaba6ee1 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.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.mllib.regression + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.Vectors + +class LabeledPointSuite extends FunSuite { + + test("parse labeled points") { + val points = Seq( + LabeledPoint(1.0, Vectors.dense(1.0, 0.0)), + LabeledPoint(0.0, Vectors.sparse(2, Array(1), Array(-1.0)))) + points.foreach { p => + assert(p === LabeledPointParser.parse(p.toString)) + } + } + + test("parse labeled points with v0.9 format") { + val point = LabeledPointParser.parse("1.0,1.0 0.0 -2.0") + assert(point === LabeledPoint(1.0, Vectors.dense(1.0, 0.0, -2.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 3d05fb68988c8..c14870fb969a8 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 @@ -160,5 +160,33 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { } } -} + test("loadVectors") { + val vectors = sc.parallelize(Seq( + Vectors.dense(1.0, 2.0), + Vectors.sparse(2, Array(1), Array(-1.0)), + Vectors.dense(0.0, 1.0) + ), 2) + val tempDir = Files.createTempDir() + val outputDir = new File(tempDir, "vectors") + val path = outputDir.toURI.toString + vectors.saveAsTextFile(path) + val loaded = loadVectors(sc, path) + assert(vectors.collect().toSet === loaded.collect().toSet) + Utils.deleteRecursively(tempDir) + } + test("loadLabeledPoints") { + val points = sc.parallelize(Seq( + LabeledPoint(1.0, Vectors.dense(1.0, 2.0)), + LabeledPoint(0.0, Vectors.sparse(2, Array(1), Array(-1.0))), + LabeledPoint(1.0, Vectors.dense(0.0, 1.0)) + ), 2) + val tempDir = Files.createTempDir() + val outputDir = new File(tempDir, "points") + val path = outputDir.toURI.toString + points.saveAsTextFile(path) + val loaded = loadLabeledPoints(sc, path) + assert(points.collect().toSet === loaded.collect().toSet) + Utils.deleteRecursively(tempDir) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/NumericParserSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/NumericParserSuite.scala new file mode 100644 index 0000000000000..f68fb95eac4e4 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/NumericParserSuite.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.mllib.util + +import org.scalatest.FunSuite + +import org.apache.spark.SparkException + +class NumericParserSuite extends FunSuite { + + test("parser") { + val s = "((1.0,2e3),-4,[5e-6,7.0E8],+9)" + val parsed = NumericParser.parse(s).asInstanceOf[Seq[_]] + assert(parsed(0).asInstanceOf[Seq[_]] === Seq(1.0, 2.0e3)) + assert(parsed(1).asInstanceOf[Double] === -4.0) + assert(parsed(2).asInstanceOf[Array[Double]] === Array(5.0e-6, 7.0e8)) + assert(parsed(3).asInstanceOf[Double] === 9.0) + + val malformatted = Seq("a", "[1,,]", "0.123.4", "1 2", "3+4") + malformatted.foreach { s => + intercept[SparkException] { + NumericParser.parse(s) + println(s"Didn't detect malformatted string $s.") + } + } + } +} diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index 802a27a8da14d..a411a5d5914e0 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -22,6 +22,7 @@ from pyspark.mllib.linalg import SparseVector from pyspark.serializers import Serializer + """ Common utilities shared throughout MLlib, primarily for dealing with different data types. These include: @@ -147,7 +148,7 @@ def _serialize_sparse_vector(v): return ba -def _deserialize_double_vector(ba): +def _deserialize_double_vector(ba, offset=0): """Deserialize a double vector from a mutually understood format. >>> x = array([1.0, 2.0, 3.0, 4.0, -1.0, 0.0, -0.0]) @@ -160,43 +161,46 @@ def _deserialize_double_vector(ba): if type(ba) != bytearray: raise TypeError("_deserialize_double_vector called on a %s; " "wanted bytearray" % type(ba)) - if len(ba) < 5: + nb = len(ba) - offset + if nb < 5: raise TypeError("_deserialize_double_vector called on a %d-byte array, " - "which is too short" % len(ba)) - if ba[0] == DENSE_VECTOR_MAGIC: - return _deserialize_dense_vector(ba) - elif ba[0] == SPARSE_VECTOR_MAGIC: - return _deserialize_sparse_vector(ba) + "which is too short" % nb) + if ba[offset] == DENSE_VECTOR_MAGIC: + return _deserialize_dense_vector(ba, offset) + elif ba[offset] == SPARSE_VECTOR_MAGIC: + return _deserialize_sparse_vector(ba, offset) else: raise TypeError("_deserialize_double_vector called on bytearray " "with wrong magic") -def _deserialize_dense_vector(ba): +def _deserialize_dense_vector(ba, offset=0): """Deserialize a dense vector into a numpy array.""" - if len(ba) < 5: + nb = len(ba) - offset + if nb < 5: raise TypeError("_deserialize_dense_vector called on a %d-byte array, " - "which is too short" % len(ba)) - length = ndarray(shape=[1], buffer=ba, offset=1, dtype=int32)[0] - if len(ba) != 8 * length + 5: + "which is too short" % nb) + length = ndarray(shape=[1], buffer=ba, offset=offset + 1, dtype=int32)[0] + if nb < 8 * length + 5: raise TypeError("_deserialize_dense_vector called on bytearray " "with wrong length") - return _deserialize_numpy_array([length], ba, 5) + return _deserialize_numpy_array([length], ba, offset + 5) -def _deserialize_sparse_vector(ba): +def _deserialize_sparse_vector(ba, offset=0): """Deserialize a sparse vector into a MLlib SparseVector object.""" - if len(ba) < 9: + nb = len(ba) - offset + if nb < 9: raise TypeError("_deserialize_sparse_vector called on a %d-byte array, " - "which is too short" % len(ba)) - header = ndarray(shape=[2], buffer=ba, offset=1, dtype=int32) + "which is too short" % nb) + header = ndarray(shape=[2], buffer=ba, offset=offset + 1, dtype=int32) size = header[0] nonzeros = header[1] - if len(ba) != 9 + 12 * nonzeros: + if nb < 9 + 12 * nonzeros: raise TypeError("_deserialize_sparse_vector called on bytearray " "with wrong length") - indices = _deserialize_numpy_array([nonzeros], ba, 9, dtype=int32) - values = _deserialize_numpy_array([nonzeros], ba, 9 + 4 * nonzeros, dtype=float64) + indices = _deserialize_numpy_array([nonzeros], ba, offset + 9, dtype=int32) + values = _deserialize_numpy_array([nonzeros], ba, offset + 9 + 4 * nonzeros, dtype=float64) return SparseVector(int(size), indices, values) @@ -243,7 +247,23 @@ def _deserialize_double_matrix(ba): def _serialize_labeled_point(p): - """Serialize a LabeledPoint with a features vector of any type.""" + """ + Serialize a LabeledPoint with a features vector of any type. + + >>> from pyspark.mllib.regression import LabeledPoint + >>> dp0 = LabeledPoint(0.5, array([1.0, 2.0, 3.0, 4.0, -1.0, 0.0, -0.0])) + >>> dp1 = _deserialize_labeled_point(_serialize_labeled_point(dp0)) + >>> dp1.label == dp0.label + True + >>> array_equal(dp1.features, dp0.features) + True + >>> sp0 = LabeledPoint(0.0, SparseVector(4, [1, 3], [3.0, 5.5])) + >>> sp1 = _deserialize_labeled_point(_serialize_labeled_point(sp0)) + >>> sp1.label == sp1.label + True + >>> sp1.features == sp0.features + True + """ from pyspark.mllib.regression import LabeledPoint serialized_features = _serialize_double_vector(p.features) header = bytearray(9) @@ -252,6 +272,16 @@ def _serialize_labeled_point(p): header_float[0] = p.label return header + serialized_features +def _deserialize_labeled_point(ba, offset=0): + """Deserialize a LabeledPoint from a mutually understood format.""" + from pyspark.mllib.regression import LabeledPoint + if type(ba) != bytearray: + raise TypeError("Expecting a bytearray but got %s" % type(ba)) + if ba[offset] != LABELED_POINT_MAGIC: + raise TypeError("Expecting magic number %d but got %d" % (LABELED_POINT_MAGIC, ba[0])) + label = ndarray(shape=[1], buffer=ba, offset=offset + 1, dtype=float64)[0] + features = _deserialize_double_vector(ba, offset + 9) + return LabeledPoint(label, features) def _copyto(array, buffer, offset, shape, dtype): """ diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 276684272068b..db39ed0acdb66 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -43,11 +43,11 @@ def __init__(self, size, *args): or two sorted lists containing indices and values. >>> print SparseVector(4, {1: 1.0, 3: 5.5}) - [1: 1.0, 3: 5.5] + (4,[1,3],[1.0,5.5]) >>> print SparseVector(4, [(1, 1.0), (3, 5.5)]) - [1: 1.0, 3: 5.5] + (4,[1,3],[1.0,5.5]) >>> print SparseVector(4, [1, 3], [1.0, 5.5]) - [1: 1.0, 3: 5.5] + (4,[1,3],[1.0,5.5]) """ self.size = int(size) assert 1 <= len(args) <= 2, "must pass either 2 or 3 arguments" @@ -160,10 +160,9 @@ def squared_distance(self, other): return result def __str__(self): - inds = self.indices - vals = self.values - entries = ", ".join(["{0}: {1}".format(inds[i], vals[i]) for i in xrange(len(inds))]) - return "[" + entries + "]" + inds = "[" + ",".join([str(i) for i in self.indices]) + "]" + vals = "[" + ",".join([str(v) for v in self.values]) + "]" + return "(" + ",".join((str(self.size), inds, vals)) + ")" def __repr__(self): inds = self.indices @@ -213,11 +212,11 @@ def sparse(size, *args): or two sorted lists containing indices and values. >>> print Vectors.sparse(4, {1: 1.0, 3: 5.5}) - [1: 1.0, 3: 5.5] + (4,[1,3],[1.0,5.5]) >>> print Vectors.sparse(4, [(1, 1.0), (3, 5.5)]) - [1: 1.0, 3: 5.5] + (4,[1,3],[1.0,5.5]) >>> print Vectors.sparse(4, [1, 3], [1.0, 5.5]) - [1: 1.0, 3: 5.5] + (4,[1,3],[1.0,5.5]) """ return SparseVector(size, *args) @@ -232,6 +231,21 @@ def dense(elements): """ return array(elements, dtype=float64) + @staticmethod + def stringify(vector): + """ + Converts a vector into a string, which can be recognized by + Vectors.parse(). + + >>> Vectors.stringify(Vectors.sparse(2, [1], [1.0])) + '(2,[1],[1.0])' + >>> Vectors.stringify(Vectors.dense([0.0, 1.0])) + '[0.0,1.0]' + """ + if type(vector) == SparseVector: + return str(vector) + else: + return "[" + ",".join([str(v) for v in vector]) + "]" def _test(): import doctest diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index bc7de6d2e8958..b84bc531dec8c 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -23,7 +23,7 @@ _serialize_double_vector, _deserialize_double_vector, \ _get_initial_weights, _serialize_rating, _regression_train_wrapper, \ _linear_predictor_typecheck, _have_scipy, _scipy_issparse -from pyspark.mllib.linalg import SparseVector +from pyspark.mllib.linalg import SparseVector, Vectors class LabeledPoint(object): @@ -44,6 +44,9 @@ def __init__(self, label, features): else: raise TypeError("Expected NumPy array, list, SparseVector, or scipy.sparse matrix") + def __str__(self): + return "(" + ",".join((str(self.label), Vectors.stringify(self.features))) + ")" + class LinearModel(object): """A linear model that has a vector of coefficients and an intercept.""" diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 0e5f4520b9402..e24c144f458bd 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -19,7 +19,10 @@ from pyspark.mllib.linalg import Vectors, SparseVector from pyspark.mllib.regression import LabeledPoint -from pyspark.mllib._common import _convert_vector +from pyspark.mllib._common import _convert_vector, _deserialize_labeled_point +from pyspark.rdd import RDD +from pyspark.serializers import NoOpSerializer + class MLUtils: @@ -105,24 +108,18 @@ def loadLibSVMFile(sc, path, multiclass=False, numFeatures=-1, minPartitions=Non >>> 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] + >>> type(examples[0]) == LabeledPoint + True + >>> print examples[0] + (1.0,(6,[0,2,4],[1.0,2.0,3.0])) + >>> type(examples[1]) == LabeledPoint + True + >>> print examples[1] + (0.0,(6,[],[])) + >>> type(examples[2]) == LabeledPoint + True + >>> print examples[2] + (0.0,(6,[1,3,5],[4.0,5.0,6.0])) >>> multiclass_examples[1].label -1.0 """ @@ -158,6 +155,40 @@ def saveAsLibSVMFile(data, dir): lines.saveAsTextFile(dir) + @staticmethod + def loadLabeledPoints(sc, path, minPartitions=None): + """ + Load labeled points saved using RDD.saveAsTextFile. + + @param sc: Spark context + @param path: file or directory path in any Hadoop-supported file + system URI + @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 + >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])), \ + LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))] + >>> tempFile = NamedTemporaryFile(delete=True) + >>> tempFile.close() + >>> sc.parallelize(examples, 1).saveAsTextFile(tempFile.name) + >>> loaded = MLUtils.loadLabeledPoints(sc, tempFile.name).collect() + >>> type(loaded[0]) == LabeledPoint + True + >>> print examples[0] + (1.1,(3,[0,2],[-1.23,4.56e-07])) + >>> type(examples[1]) == LabeledPoint + True + >>> print examples[1] + (0.0,[1.01,2.02,3.03]) + """ + minPartitions = minPartitions or min(sc.defaultParallelism, 2) + jSerialized = sc._jvm.PythonMLLibAPI().loadLabeledPoints(sc._jsc, path, minPartitions) + serialized = RDD(jSerialized, sc, NoOpSerializer()) + return serialized.map(lambda bytes: _deserialize_labeled_point(bytearray(bytes))) + + def _test(): import doctest from pyspark.context import SparkContext From 1765c8d0ddf6bb5bc3c21f994456eba04c581de4 Mon Sep 17 00:00:00 2001 From: Colin McCabe Date: Wed, 4 Jun 2014 15:56:29 -0700 Subject: [PATCH 23/54] SPARK-1518: FileLogger: Fix compile against Hadoop trunk In Hadoop trunk (currently Hadoop 3.0.0), the deprecated FSDataOutputStream#sync() method has been removed. Instead, we should call FSDataOutputStream#hflush, which does the same thing as the deprecated method used to do. Author: Colin McCabe Closes #898 from cmccabe/SPARK-1518 and squashes the following commits: 752b9d7 [Colin McCabe] FileLogger: Fix compile against Hadoop trunk --- .../scala/org/apache/spark/util/FileLogger.scala | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) 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 0e6d21b22023a..6a95dc06e155d 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -61,6 +61,14 @@ private[spark] class FileLogger( // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None + // The Hadoop APIs have changed over time, so we use reflection to figure out + // the correct method to use to flush a hadoop data stream. See SPARK-1518 + // for details. + private val hadoopFlushMethod = { + val cls = classOf[FSDataOutputStream] + scala.util.Try(cls.getMethod("hflush")).getOrElse(cls.getMethod("sync")) + } + private var writer: Option[PrintWriter] = None /** @@ -149,13 +157,13 @@ private[spark] class FileLogger( /** * Flush the writer to disk manually. * - * If the Hadoop FileSystem is used, the underlying FSDataOutputStream (r1.0.4) must be - * sync()'ed manually as it does not support flush(), which is invoked by when higher - * level streams are flushed. + * When using a Hadoop filesystem, we need to invoke the hflush or sync + * method. In HDFS, hflush guarantees that the data gets to all the + * DataNodes. */ def flush() { writer.foreach(_.flush()) - hadoopDataStream.foreach(_.sync()) + hadoopDataStream.foreach(hadoopFlushMethod.invoke(_)) } /** From 11ded3f66f178e4d8d2b23491dd5e0ea23bcf719 Mon Sep 17 00:00:00 2001 From: Varakhedi Sujeet Date: Wed, 4 Jun 2014 16:01:56 -0700 Subject: [PATCH 24/54] SPARK-1790: Update EC2 scripts to support r3 instance types Author: Varakhedi Sujeet Closes #960 from sujeetv/ec2-r3 and squashes the following commits: 3cb9fd5 [Varakhedi Sujeet] SPARK-1790: Update EC2 scripts to support r3 instance --- ec2/spark_ec2.py | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 3af9f66e17dc2..9d5748ba4bc23 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -230,7 +230,12 @@ def get_spark_ami(opts): "c3.xlarge": "pvm", "c3.2xlarge": "pvm", "c3.4xlarge": "pvm", - "c3.8xlarge": "pvm" + "c3.8xlarge": "pvm", + "r3.large": "hvm", + "r3.xlarge": "hvm", + "r3.2xlarge": "hvm", + "r3.4xlarge": "hvm", + "r3.8xlarge": "hvm" } if opts.instance_type in instance_types: instance_type = instance_types[opts.instance_type] @@ -538,7 +543,12 @@ def get_num_disks(instance_type): "c3.xlarge": 2, "c3.2xlarge": 2, "c3.4xlarge": 2, - "c3.8xlarge": 2 + "c3.8xlarge": 2, + "r3.large": 1, + "r3.xlarge": 1, + "r3.2xlarge": 1, + "r3.4xlarge": 1, + "r3.8xlarge": 2 } if instance_type in disks_by_instance: return disks_by_instance[instance_type] From abea2d4ff099036c67fc73136d0e61d0d0e22123 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 4 Jun 2014 16:45:53 -0700 Subject: [PATCH 25/54] Minor: Fix documentation error from apache/spark#946 Author: Ankur Dave Closes #970 from ankurdave/SPARK-1991_docfix and squashes the following commits: 6d07343 [Ankur Dave] Minor: Fix documentation error from apache/spark#946 --- .../src/main/scala/org/apache/spark/graphx/GraphLoader.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 2e814e34f9ad8..f4c79365b16da 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -49,8 +49,8 @@ object GraphLoader extends Logging { * @param canonicalOrientation whether to orient edges in the positive * direction * @param minEdgePartitions the number of partitions for the edge RDD - * @param edgeStorageLevel the desired storage level for the edge partitions. To set the vertex - * storage level, call [[org.apache.spark.graphx.Graph#persistVertices]]. + * @param edgeStorageLevel the desired storage level for the edge partitions + * @param vertexStorageLevel the desired storage level for the vertex partitions */ def edgeListFile( sc: SparkContext, From b77c19be053125fde99b098ec1e1162f25b5433c Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 4 Jun 2014 22:56:49 -0700 Subject: [PATCH 26/54] Fix issue in ReplSuite with hadoop-provided profile. When building the assembly with the maven "hadoop-provided" profile, the executors were failing to come up because Hadoop classes were not found in the classpath anymore; so add them explicitly to the classpath using spark.executor.extraClassPath. This is only needed for the local-cluster mode, but doesn't affect other tests, so it's added for all of them to keep the code simpler. Author: Marcelo Vanzin Closes #781 from vanzin/repl-test-fix and squashes the following commits: 4f0a3b0 [Marcelo Vanzin] Fix issue in ReplSuite with hadoop-provided profile. --- .../scala/org/apache/spark/repl/ReplSuite.scala | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) 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 98cdfd0054713..7c765edd55027 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -32,6 +32,8 @@ import org.apache.spark.util.Utils class ReplSuite extends FunSuite { def runInterpreter(master: String, input: String): String = { + val CONF_EXECUTOR_CLASSPATH = "spark.executor.extraClassPath" + val in = new BufferedReader(new StringReader(input + "\n")) val out = new StringWriter() val cl = getClass.getClassLoader @@ -44,13 +46,23 @@ class ReplSuite extends FunSuite { } } } + val classpath = paths.mkString(File.pathSeparator) + + val oldExecutorClasspath = System.getProperty(CONF_EXECUTOR_CLASSPATH) + System.setProperty(CONF_EXECUTOR_CLASSPATH, classpath) + val interp = new SparkILoop(in, new PrintWriter(out), master) org.apache.spark.repl.Main.interp = interp - interp.process(Array("-classpath", paths.mkString(File.pathSeparator))) + interp.process(Array("-classpath", classpath)) org.apache.spark.repl.Main.interp = null if (interp.sparkContext != null) { interp.sparkContext.stop() } + if (oldExecutorClasspath != null) { + System.setProperty(CONF_EXECUTOR_CLASSPATH, oldExecutorClasspath) + } else { + System.clearProperty(CONF_EXECUTOR_CLASSPATH) + } return out.toString } From 7c160293d6d708718d566e700cfb407a31280b89 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 5 Jun 2014 11:27:33 -0700 Subject: [PATCH 27/54] [SPARK-2029] Bump pom.xml version number of master branch to 1.1.0-SNAPSHOT. Author: Takuya UESHIN Closes #974 from ueshin/issues/SPARK-2029 and squashes the following commits: e19e8f4 [Takuya UESHIN] Bump version number to 1.1.0-SNAPSHOT. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/java8-tests/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/alpha/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 23 files changed, 23 insertions(+), 23 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 963357b9ab167..0c60b66c3daca 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 355f437c5b16a..c8e39a415af28 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 0777c5b1f03d4..0c746175afa73 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 874bcd7916f35..4f6d7fdb87d47 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 6aec215687fe0..c1f581967777b 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 979eb0ca624bd..d014a7aad0fca 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 7b2dc5ba1d7f9..4980208cba3b0 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 5766d3a0d44ec..7073bd4404d9c 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 4ed4196bd8662..cf306e0dca8bd 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml index 602f66f9c5cf1..955ec1a8c3033 100644 --- a/extras/java8-tests/pom.xml +++ b/extras/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 11ac827ed54a0..22ea330b4374d 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index dc108d2fe7fbd..7d5d83e7f3bb9 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index cdd33dbb7970d..4aae2026dcaf2 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index fcd6f66b4414a..87c8e29ad1069 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index bcdb24b040cc8..4a66408ef3d2d 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 8d2e4baf69e30..6c78c34486010 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index fb3b190b4ec5a..e65ca6be485e3 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 9254b70e64a08..5ede76e5c3904 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 6435224a14674..f506d6ce34a6f 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 1875c497bc61c..79cd8551d0722 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index e076ca1d44b97..b8a631dd0bb3b 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 2811ffffbdfa2..ef7066ef1fdfc 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 0780f251b595c..0931beb505508 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml From 89cdbb087cb2f0d03be2dd77440300c6bd61c792 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 5 Jun 2014 11:39:35 -0700 Subject: [PATCH 28/54] SPARK-1677: allow user to disable output dir existence checking https://issues.apache.org/jira/browse/SPARK-1677 For compatibility with older versions of Spark it would be nice to have an option `spark.hadoop.validateOutputSpecs` (default true) for the user to disable the output directory existence checking Author: CodingCat Closes #947 from CodingCat/SPARK-1677 and squashes the following commits: 7930f83 [CodingCat] miao c0c0e03 [CodingCat] bug fix and doc update 5318562 [CodingCat] bug fix 13219b5 [CodingCat] allow user to disable output dir existence checking --- .../apache/spark/rdd/PairRDDFunctions.scala | 6 +++-- .../scala/org/apache/spark/FileSuite.scala | 22 +++++++++++++++++++ docs/configuration.md | 8 +++++++ 3 files changed, 34 insertions(+), 2 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 f2ce3cbd47f93..8909980957058 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -737,7 +737,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val outfmt = job.getOutputFormatClass val jobFormat = outfmt.newInstance - if (jobFormat.isInstanceOf[NewFileOutputFormat[_, _]]) { + if (self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true) && + jobFormat.isInstanceOf[NewFileOutputFormat[_, _]]) { // FileOutputFormat ignores the filesystem parameter jobFormat.checkOutputSpecs(job) } @@ -803,7 +804,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + valueClass.getSimpleName + ")") - if (outputFormatInstance.isInstanceOf[FileOutputFormat[_, _]]) { + if (self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true) && + outputFormatInstance.isInstanceOf[FileOutputFormat[_, _]]) { // FileOutputFormat ignores the filesystem parameter val ignoredFs = FileSystem.get(conf) conf.getOutputFormat.checkOutputSpecs(ignoredFs, conf) diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 1f2206b1f0379..070e974657860 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -230,6 +230,17 @@ class FileSuite extends FunSuite with LocalSparkContext { } } + test ("allow user to disable the output directory existence checking (old Hadoop API") { + val sf = new SparkConf() + sf.setAppName("test").setMaster("local").set("spark.hadoop.validateOutputSpecs", "false") + sc = new SparkContext(sf) + 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) + } + test ("prevent user from overwriting the empty directory (new Hadoop API)") { sc = new SparkContext("local", "test") val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) @@ -248,6 +259,17 @@ class FileSuite extends FunSuite with LocalSparkContext { } } + test ("allow user to disable the output directory existence checking (new Hadoop API") { + val sf = new SparkConf() + sf.setAppName("test").setMaster("local").set("spark.hadoop.validateOutputSpecs", "false") + sc = new SparkContext(sf) + 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) + } + test ("save Hadoop Dataset through old Hadoop API") { sc = new SparkContext("local", "test") val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) diff --git a/docs/configuration.md b/docs/configuration.md index 0697f7fc2fd91..71fafa573467f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -487,6 +487,14 @@ Apart from these, the following properties are also available, and may be useful this duration will be cleared as well. + + spark.hadoop.validateOutputSpecs + true + If set to true, validates the output specification (e.g. checking if the output directory already exists) + used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing + output directories. We recommend that users do not disable this except if trying to achieve compatibility with + previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. + #### Networking From e4c11eef2f64df0b6a432f40b669486d91ca6352 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 5 Jun 2014 12:00:31 -0700 Subject: [PATCH 29/54] [SPARK-2036] [SQL] CaseConversionExpression should check if the evaluated value is null. `CaseConversionExpression` should check if the evaluated value is `null`. Author: Takuya UESHIN Closes #982 from ueshin/issues/SPARK-2036 and squashes the following commits: 61e1c54 [Takuya UESHIN] Add check if the evaluated value is null. --- .../catalyst/expressions/stringOperations.scala | 8 ++++++-- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 14 ++++++++++++++ .../test/scala/org/apache/spark/sql/TestData.scala | 8 ++++++++ 3 files changed, 28 insertions(+), 2 deletions(-) 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 dcded0774180e..420303408451f 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 @@ -81,8 +81,12 @@ trait CaseConversionExpression { def dataType: DataType = StringType override def eval(input: Row): Any = { - val converted = child.eval(input) - convert(converted.toString) + val evaluated = child.eval(input) + if (evaluated == null) { + null + } else { + convert(evaluated.toString) + } } } 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 95860e6683f67..e2ad3915d3134 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 @@ -322,6 +322,13 @@ class SQLQuerySuite extends QueryTest { (2, "B"), (3, "C"), (4, "D"))) + + checkAnswer( + sql("SELECT n, UPPER(s) FROM nullStrings"), + Seq( + (1, "ABC"), + (2, "ABC"), + (3, null))) } test("system function lower()") { @@ -334,6 +341,13 @@ class SQLQuerySuite extends QueryTest { (4, "d"), (5, "e"), (6, "f"))) + + checkAnswer( + sql("SELECT n, LOWER(s) FROM nullStrings"), + Seq( + (1, "abc"), + (2, "abc"), + (3, null))) } } 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 944f520e43515..876bd1636aab3 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 @@ -106,4 +106,12 @@ object TestData { NullInts(null) :: Nil ) nullInts.registerAsTable("nullInts") + + case class NullStrings(n: Int, s: String) + val nullStrings = + TestSQLContext.sparkContext.parallelize( + NullStrings(1, "abc") :: + NullStrings(2, "ABC") :: + NullStrings(3, null) :: Nil) + nullStrings.registerAsTable("nullStrings") } From f6143f127db59e7f5a00fd70605f85248869347d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 5 Jun 2014 13:06:46 -0700 Subject: [PATCH 30/54] HOTFIX: Remove generated-mima-excludes file after runing MIMA. This has been causing some false failures on PR's that don't merge correctly. Author: Patrick Wendell Closes #971 from pwendell/mima and squashes the following commits: 1dc80aa [Patrick Wendell] HOTFIX: Remove generated-mima-excludes file after runing MIMA. --- dev/mima | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/mima b/dev/mima index d4099990254cc..ab6bd4469b0e8 100755 --- a/dev/mima +++ b/dev/mima @@ -31,4 +31,5 @@ if [ $ret_val != 0 ]; then echo "NOTE: Exceptions to binary compatibility can be added in project/MimaExcludes.scala" fi +rm -f .generated-mima-excludes exit $ret_val From 5473aa7c02916022430493637b1492554b48c30b Mon Sep 17 00:00:00 2001 From: Kalpit Shah Date: Thu, 5 Jun 2014 13:07:26 -0700 Subject: [PATCH 31/54] sbt 0.13.X should be using sbt-assembly 0.11.X https://github.com/sbt/sbt-assembly/blob/master/README.md Author: Kalpit Shah Closes #555 from kalpit/upgrade/sbtassembly and squashes the following commits: 1fa7324 [Kalpit Shah] sbt 0.13.X should be using sbt-assembly 0.11.X --- project/plugins.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/plugins.sbt b/project/plugins.sbt index 0cd16fd5bedd4..472819b9fb8ba 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -4,7 +4,7 @@ resolvers += Resolver.url("artifactory", url("http://scalasbt.artifactoryonline. resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/" -addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.10.2") +addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.11.2") addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0") From 668cb1defe735add91f4a5b7b8ebe7cfd5640b25 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 5 Jun 2014 13:13:33 -0700 Subject: [PATCH 32/54] Remove compile-scoped junit dependency. This avoids having junit classes showing up in the assembly jar. I verified that only test classes in the jtransforms package use junit. Author: Marcelo Vanzin Closes #794 from vanzin/junit-dep-exclusion and squashes the following commits: 274e1c2 [Marcelo Vanzin] Remove junit from assembly in sbt build also. ad950be [Marcelo Vanzin] Remove compile-scoped junit dependency. --- mllib/pom.xml | 8 ++++++++ project/SparkBuild.scala | 3 ++- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/mllib/pom.xml b/mllib/pom.xml index 4aae2026dcaf2..878cb83dbf783 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -50,6 +50,14 @@ org.scalanlp breeze_${scala.binary.version} 0.7 + + + + junit + junit + + org.scalatest diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index efb0b9319be13..d0049a8ac43aa 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -322,6 +322,7 @@ object SparkBuild extends Build { val excludeJruby = ExclusionRule(organization = "org.jruby") val excludeThrift = ExclusionRule(organization = "org.apache.thrift") val excludeServletApi = ExclusionRule(organization = "javax.servlet", artifact = "servlet-api") + val excludeJUnit = ExclusionRule(organization = "junit") def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark", version: String = "1.0.0", crossVersion: String = "2.10"): Option[sbt.ModuleID] = { @@ -466,7 +467,7 @@ object SparkBuild extends Build { previousArtifact := sparkPreviousArtifact("spark-mllib"), libraryDependencies ++= Seq( "org.jblas" % "jblas" % jblasVersion, - "org.scalanlp" %% "breeze" % "0.7" + "org.scalanlp" %% "breeze" % "0.7" excludeAll(excludeJUnit) ) ) From c7a183b2c2bca13565496495b4ae3a3a9f63f9ab Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 5 Jun 2014 17:42:08 -0700 Subject: [PATCH 33/54] [SPARK-2041][SQL] Correctly analyze queries where columnName == tableName. Author: Michael Armbrust Closes #985 from marmbrus/tableName and squashes the following commits: 3caaa27 [Michael Armbrust] Correctly analyze queries where columnName == tableName. --- .../spark/sql/catalyst/plans/logical/LogicalPlan.scala | 3 ++- .../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 6 ++++++ sql/core/src/test/scala/org/apache/spark/sql/TestData.scala | 3 +++ 3 files changed, 11 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 5eb52d5350f55..2b8fbdcde9d37 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -64,7 +64,8 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { // struct fields. val options = children.flatMap(_.output).flatMap { option => // If the first part of the desired name matches a qualifier for this possible match, drop it. - val remainingParts = if (option.qualifiers contains parts.head) parts.drop(1) else parts + val remainingParts = + if (option.qualifiers.contains(parts.head) && parts.size > 1) parts.drop(1) else parts if (option.name == remainingParts.head) (option, remainingParts.tail.toList) :: Nil else Nil } 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 e2ad3915d3134..aa0c426f6fcb3 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,12 @@ class SQLQuerySuite extends QueryTest { // Make sure the tables are loaded. TestData + test("SPARK-2041 column name equals tablename") { + checkAnswer( + sql("SELECT tableName FROM tableName"), + "test") + } + test("index into array") { checkAnswer( sql("SELECT data, data[0], data[0] + data[1], data[0 + 1] FROM arrayData"), 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 876bd1636aab3..05de736bbce1b 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 @@ -114,4 +114,7 @@ object TestData { NullStrings(2, "ABC") :: NullStrings(3, null) :: Nil) nullStrings.registerAsTable("nullStrings") + + case class TableName(tableName: String) + TestSQLContext.sparkContext.parallelize(TableName("test") :: Nil).registerAsTable("tableName") } From 3d3f8c8004da110ca97973119e9d9f04f878ee81 Mon Sep 17 00:00:00 2001 From: CrazyJvm Date: Thu, 5 Jun 2014 17:44:46 -0700 Subject: [PATCH 34/54] Use pluggable clock in DAGSheduler #SPARK-2031 DAGScheduler supports pluggable clock like what TaskSetManager does. Author: CrazyJvm Closes #976 from CrazyJvm/clock and squashes the following commits: 6779a4c [CrazyJvm] Use pluggable clock in DAGSheduler --- .../org/apache/spark/scheduler/DAGScheduler.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 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 ccff6a3d1aebc..e09a4221e8315 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -38,7 +38,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.Utils +import org.apache.spark.util.{SystemClock, Clock, Utils} /** * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of @@ -61,7 +61,8 @@ class DAGScheduler( listenerBus: LiveListenerBus, mapOutputTracker: MapOutputTrackerMaster, blockManagerMaster: BlockManagerMaster, - env: SparkEnv) + env: SparkEnv, + clock: Clock = SystemClock) extends Logging { import DAGScheduler._ @@ -781,7 +782,7 @@ class DAGScheduler( logDebug("New pending tasks: " + myPending) taskScheduler.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) - stageToInfos(stage).submissionTime = Some(System.currentTimeMillis()) + stageToInfos(stage).submissionTime = Some(clock.getTime()) } else { logDebug("Stage " + stage + " is actually done; %b %d %d".format( stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions)) @@ -807,11 +808,11 @@ class DAGScheduler( def markStageAsFinished(stage: Stage) = { val serviceTime = stageToInfos(stage).submissionTime match { - case Some(t) => "%.03f".format((System.currentTimeMillis() - t) / 1000.0) + case Some(t) => "%.03f".format((clock.getTime() - t) / 1000.0) case _ => "Unknown" } logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) - stageToInfos(stage).completionTime = Some(System.currentTimeMillis()) + stageToInfos(stage).completionTime = Some(clock.getTime()) listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) runningStages -= stage } @@ -1015,7 +1016,7 @@ class DAGScheduler( return } val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq - stageToInfos(failedStage).completionTime = Some(System.currentTimeMillis()) + stageToInfos(failedStage).completionTime = Some(clock.getTime()) for (resultStage <- dependentStages) { val job = resultStageToJob(resultStage) failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason", From 9bad0b73722fb359f14db864e69aa7efde3588c5 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 5 Jun 2014 17:45:38 -0700 Subject: [PATCH 35/54] [SPARK-2025] Unpersist edges of previous graph in Pregel Due to a bug introduced by apache/spark#497, Pregel does not unpersist replicated vertices from previous iterations. As a result, they stay cached until memory is full, wasting GC time. This PR corrects the problem by unpersisting both the edges and the replicated vertices of previous iterations. This is safe because the edges and replicated vertices of the current iteration are cached by the call to `g.cache()` and then materialized by the call to `messages.count()`. Therefore no unmaterialized RDDs depend on `prevG.edges`. I verified that no recomputation occurs by running PageRank with a custom patch to Spark that warns when a partition is recomputed. Thanks to Tim Weninger for reporting this bug. Author: Ankur Dave Closes #972 from ankurdave/SPARK-2025 and squashes the following commits: 13d5b07 [Ankur Dave] Unpersist edges of previous graph in Pregel --- graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala | 1 + 1 file changed, 1 insertion(+) 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 4572eab2875bb..5e55620147df8 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -150,6 +150,7 @@ object Pregel extends Logging { oldMessages.unpersist(blocking=false) newVerts.unpersist(blocking=false) prevG.unpersistVertices(blocking=false) + prevG.edges.unpersist(blocking=false) // count the iteration i += 1 } From b45c13e7d798f97b92f1a6329528191b8d779c4f Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 5 Jun 2014 23:01:48 -0700 Subject: [PATCH 36/54] SPARK-2043: ExternalAppendOnlyMap doesn't always find matching keys The current implementation reads one key with the next hash code as it finishes reading the keys with the current hash code, which may cause it to miss some matches of the next key. This can cause operations like join to give the wrong result when reduce tasks spill to disk and there are hash collisions, as values won't be matched together. This PR fixes it by not reading in that next key, using a peeking iterator instead. Author: Matei Zaharia Closes #986 from mateiz/spark-2043 and squashes the following commits: 0959514 [Matei Zaharia] Added unit test for having many hash collisions 892debb [Matei Zaharia] SPARK-2043: don't read a key with the next hash code in ExternalAppendOnlyMap, instead use a buffered iterator to only read values with the current hash code. --- .../collection/ExternalAppendOnlyMap.scala | 10 +++-- .../ExternalAppendOnlyMapSuite.scala | 39 ++++++++++++++++++- 2 files changed, 44 insertions(+), 5 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 170f09be21534..288badd3160f8 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 @@ -20,6 +20,7 @@ package org.apache.spark.util.collection import java.io.{InputStream, BufferedInputStream, FileInputStream, File, Serializable, EOFException} import java.util.Comparator +import scala.collection.BufferedIterator import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -231,7 +232,7 @@ class ExternalAppendOnlyMap[K, V, C]( // Input streams are derived both from the in-memory map and spilled maps on disk // The in-memory map is sorted in place, while the spilled maps are already in sorted order private val sortedMap = currentMap.destructiveSortedIterator(comparator) - private val inputStreams = Seq(sortedMap) ++ spilledMaps + private val inputStreams = (Seq(sortedMap) ++ spilledMaps).map(it => it.buffered) inputStreams.foreach { it => val kcPairs = getMorePairs(it) @@ -246,13 +247,13 @@ class ExternalAppendOnlyMap[K, V, C]( * In the event of key hash collisions, this ensures no pairs are hidden from being merged. * Assume the given iterator is in sorted order. */ - private def getMorePairs(it: Iterator[(K, C)]): ArrayBuffer[(K, C)] = { + private def getMorePairs(it: BufferedIterator[(K, C)]): ArrayBuffer[(K, C)] = { val kcPairs = new ArrayBuffer[(K, C)] if (it.hasNext) { var kc = it.next() kcPairs += kc val minHash = kc._1.hashCode() - while (it.hasNext && kc._1.hashCode() == minHash) { + while (it.hasNext && it.head._1.hashCode() == minHash) { kc = it.next() kcPairs += kc } @@ -325,7 +326,8 @@ class ExternalAppendOnlyMap[K, V, C]( * * StreamBuffers are ordered by the minimum key hash found across all of their own pairs. */ - private case class StreamBuffer(iterator: Iterator[(K, C)], pairs: ArrayBuffer[(K, C)]) + private class StreamBuffer( + val iterator: BufferedIterator[(K, C)], val pairs: ArrayBuffer[(K, C)]) extends Comparable[StreamBuffer] { def isEmpty = pairs.length == 0 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 cdebefb67510c..deb780953579d 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 @@ -277,6 +277,11 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { ("pomatoes", "eructation") // 568647356 ) + collisionPairs.foreach { case (w1, w2) => + // String.hashCode is documented to use a specific algorithm, but check just in case + assert(w1.hashCode === w2.hashCode) + } + (1 to 100000).map(_.toString).foreach { i => map.insert(i, i) } collisionPairs.foreach { case (w1, w2) => map.insert(w1, w2) @@ -296,7 +301,32 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { assert(kv._2.equals(expectedValue)) count += 1 } - assert(count == 100000 + collisionPairs.size * 2) + assert(count === 100000 + collisionPairs.size * 2) + } + + test("spilling with many hash collisions") { + val conf = new SparkConf(true) + conf.set("spark.shuffle.memoryFraction", "0.0001") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) + + val map = new ExternalAppendOnlyMap[FixedHashObject, Int, Int](_ => 1, _ + _, _ + _) + + // Insert 10 copies each of lots of objects whose hash codes are either 0 or 1. This causes + // problems if the map fails to group together the objects with the same code (SPARK-2043). + for (i <- 1 to 10) { + for (j <- 1 to 10000) { + map.insert(FixedHashObject(j, j % 2), 1) + } + } + + val it = map.iterator + var count = 0 + while (it.hasNext) { + val kv = it.next() + assert(kv._2 === 10) + count += 1 + } + assert(count === 10000) } test("spilling with hash collisions using the Int.MaxValue key") { @@ -317,3 +347,10 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } } } + +/** + * A dummy class that always returns the same hash code, to easily test hash collisions + */ +case class FixedHashObject(val v: Int, val h: Int) extends Serializable { + override def hashCode(): Int = h +} From 41db44c428a10f4453462d002d226798bb8fbdda Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 5 Jun 2014 23:20:59 -0700 Subject: [PATCH 37/54] [SPARK-2050][SQL] LIKE, RLIKE and IN in HQL should not be case sensitive. Author: Michael Armbrust Closes #989 from marmbrus/caseSensitiveFuncitons and squashes the following commits: 681de54 [Michael Armbrust] LIKE, RLIKE and IN in HQL should not be case sensitive. --- .../scala/org/apache/spark/sql/hive/HiveQl.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) 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 e8a3ee5535b6e..c133bf2423190 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 @@ -791,6 +791,10 @@ private[hive] object HiveQl { val NOT = "(?i)NOT".r val TRUE = "(?i)TRUE".r val FALSE = "(?i)FALSE".r + val LIKE = "(?i)LIKE".r + val RLIKE = "(?i)RLIKE".r + val REGEXP = "(?i)REGEXP".r + val IN = "(?i)IN".r protected def nodeToExpr(node: Node): Expression = node match { /* Attribute References */ @@ -871,14 +875,14 @@ private[hive] object HiveQl { case Token(">=", left :: right:: Nil) => GreaterThanOrEqual(nodeToExpr(left), nodeToExpr(right)) case Token("<", left :: right:: Nil) => LessThan(nodeToExpr(left), nodeToExpr(right)) case Token("<=", left :: right:: Nil) => LessThanOrEqual(nodeToExpr(left), nodeToExpr(right)) - case Token("LIKE", left :: right:: Nil) => Like(nodeToExpr(left), nodeToExpr(right)) - case Token("RLIKE", left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) - case Token("REGEXP", left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) + case Token(LIKE(), left :: right:: Nil) => Like(nodeToExpr(left), nodeToExpr(right)) + case Token(RLIKE(), left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) + case Token(REGEXP(), left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) case Token("TOK_FUNCTION", Token("TOK_ISNOTNULL", Nil) :: child :: Nil) => IsNotNull(nodeToExpr(child)) case Token("TOK_FUNCTION", Token("TOK_ISNULL", Nil) :: child :: Nil) => IsNull(nodeToExpr(child)) - case Token("TOK_FUNCTION", Token("IN", Nil) :: value :: list) => + case Token("TOK_FUNCTION", Token(IN(), Nil) :: value :: list) => In(nodeToExpr(value), list.map(nodeToExpr)) case Token("TOK_FUNCTION", Token("between", Nil) :: From 8d85359f84cc67996b4bcf1670a8a98ab4f914a2 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 5 Jun 2014 23:33:12 -0700 Subject: [PATCH 38/54] [SPARK-1552] Fix type comparison bug in {map,outerJoin}Vertices In GraphImpl, mapVertices and outerJoinVertices use a more efficient implementation when the map function conserves vertex attribute types. This is implemented by comparing the ClassTags of the old and new vertex attribute types. However, ClassTags store erased types, so the comparison will return a false positive for types with different type parameters, such as Option[Int] and Option[Double]. This PR resolves the problem by requesting that the compiler generate evidence of equality between the old and new vertex attribute types, and providing a default value for the evidence parameter if the two types are not equal. The methods can then check the value of the evidence parameter to see whether the types are equal. It also adds a test called "mapVertices changing type with same erased type" that failed before the PR and succeeds now. Callers of mapVertices and outerJoinVertices can no longer use a wildcard for a graph's VD type. To avoid "Error occurred in an application involving default arguments," they must bind VD to a type parameter, as this PR does for ShortestPaths and LabelPropagation. Author: Ankur Dave Closes #967 from ankurdave/SPARK-1552 and squashes the following commits: 68a4fff [Ankur Dave] Undo conserve naming 7388705 [Ankur Dave] Remove unnecessary ClassTag for VD parameters a704e5f [Ankur Dave] Use type equality constraint with default argument 29a5ab7 [Ankur Dave] Add failing test f458c83 [Ankur Dave] Revert "[SPARK-1552] Fix type comparison bug in mapVertices and outerJoinVertices" 16d6af8 [Ankur Dave] [SPARK-1552] Fix type comparison bug in mapVertices and outerJoinVertices --- .../scala/org/apache/spark/graphx/Graph.scala | 5 ++-- .../apache/spark/graphx/impl/GraphImpl.scala | 14 ++++++++--- .../spark/graphx/lib/LabelPropagation.scala | 2 +- .../spark/graphx/lib/ShortestPaths.scala | 2 +- .../org/apache/spark/graphx/GraphSuite.scala | 25 +++++++++++++++++++ 5 files changed, 40 insertions(+), 8 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 14ae50e6657fd..4db45c9af8fae 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -138,7 +138,8 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * }}} * */ - def mapVertices[VD2: ClassTag](map: (VertexId, VD) => VD2): Graph[VD2, ED] + def mapVertices[VD2: ClassTag](map: (VertexId, VD) => VD2) + (implicit eq: VD =:= VD2 = null): Graph[VD2, ED] /** * Transforms each edge attribute in the graph using the map function. The map function is not @@ -348,7 +349,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * }}} */ def outerJoinVertices[U: ClassTag, VD2: ClassTag](other: RDD[(VertexId, U)]) - (mapFunc: (VertexId, VD, Option[U]) => VD2) + (mapFunc: (VertexId, VD, Option[U]) => VD2)(implicit eq: VD =:= VD2 = null) : Graph[VD2, ED] /** 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 15ea05cbe281d..ccdaa82eb9162 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 @@ -104,8 +104,11 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( new GraphImpl(vertices.reverseRoutingTables(), replicatedVertexView.reverse()) } - override def mapVertices[VD2: ClassTag](f: (VertexId, VD) => VD2): Graph[VD2, ED] = { - if (classTag[VD] equals classTag[VD2]) { + override def mapVertices[VD2: ClassTag] + (f: (VertexId, VD) => VD2)(implicit eq: VD =:= VD2 = null): Graph[VD2, ED] = { + // The implicit parameter eq will be populated by the compiler if VD and VD2 are equal, and left + // null if not + if (eq != null) { vertices.cache() // The map preserves type, so we can use incremental replication val newVerts = vertices.mapVertexPartitions(_.map(f)).cache() @@ -232,8 +235,11 @@ 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] = { - if (classTag[VD] equals classTag[VD2]) { + (updateF: (VertexId, VD, Option[U]) => VD2) + (implicit eq: VD =:= VD2 = null): Graph[VD2, ED] = { + // The implicit parameter eq will be populated by the compiler if VD and VD2 are equal, and left + // null if not + if (eq != null) { vertices.cache() // updateF preserves type, so we can use incremental replication val newVerts = vertices.leftJoin(other)(updateF).cache() 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 index 776bfb8dd6bfa..82e9e06515179 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala @@ -41,7 +41,7 @@ object LabelPropagation { * * @return a graph with vertex attributes containing the label of community affiliation */ - def run[ED: ClassTag](graph: Graph[_, ED], maxSteps: Int): Graph[VertexId, ED] = { + def run[VD, ED: ClassTag](graph: Graph[VD, 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))) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ShortestPaths.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ShortestPaths.scala index bba070f256d80..590f0474957dd 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/ShortestPaths.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ShortestPaths.scala @@ -49,7 +49,7 @@ object ShortestPaths { * @return a graph where each vertex attribute is a map containing the shortest-path distance to * each reachable landmark vertex. */ - def run[ED: ClassTag](graph: Graph[_, ED], landmarks: Seq[VertexId]): Graph[SPMap, ED] = { + def run[VD, ED: ClassTag](graph: Graph[VD, ED], landmarks: Seq[VertexId]): Graph[SPMap, ED] = { val spGraph = graph.mapVertices { (vid, attr) => if (landmarks.contains(vid)) makeMap(vid -> 0) else makeMap() } 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 abc25d0671133..6506bac73d71c 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -159,6 +159,31 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } + test("mapVertices changing type with same erased type") { + withSpark { sc => + val vertices = sc.parallelize(Array[(Long, Option[java.lang.Integer])]( + (1L, Some(1)), + (2L, Some(2)), + (3L, Some(3)) + )) + val edges = sc.parallelize(Array( + Edge(1L, 2L, 0), + Edge(2L, 3L, 0), + Edge(3L, 1L, 0) + )) + val graph0 = Graph(vertices, edges) + // Trigger initial vertex replication + graph0.triplets.foreach(x => {}) + // Change type of replicated vertices, but preserve erased type + val graph1 = graph0.mapVertices { + case (vid, integerOpt) => integerOpt.map((x: java.lang.Integer) => (x.toDouble): java.lang.Double) + } + // Access replicated vertices, exposing the erased type + val graph2 = graph1.mapTriplets(t => t.srcAttr.get) + assert(graph2.edges.map(_.attr).collect.toSet === Set[java.lang.Double](1.0, 2.0, 3.0)) + } + } + test("mapEdges") { withSpark { sc => val n = 3 From 8d210560be8b143e48abfbaca347f383b5aa4798 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 6 Jun 2014 11:31:37 -0700 Subject: [PATCH 39/54] [SPARK-2050 - 2][SQL] DIV and BETWEEN should not be case sensitive. Followup: #989 Author: Michael Armbrust Closes #994 from marmbrus/caseSensitiveFunctions2 and squashes the following commits: 9d9c8ed [Michael Armbrust] Fix DIV and BETWEEN. --- .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 6 ++++-- .../golden/between-0-570e4c4bf2968368ac2f325c6c940a06 | 1 + .../resources/golden/div-0-5e7fc5719c7265bc1d6af52005ebad03 | 1 + .../apache/spark/sql/hive/execution/HiveQuerySuite.scala | 6 ++++-- 4 files changed, 10 insertions(+), 4 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/between-0-570e4c4bf2968368ac2f325c6c940a06 create mode 100644 sql/hive/src/test/resources/golden/div-0-5e7fc5719c7265bc1d6af52005ebad03 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 c133bf2423190..7e91c16c6b93a 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 @@ -795,6 +795,8 @@ private[hive] object HiveQl { val RLIKE = "(?i)RLIKE".r val REGEXP = "(?i)REGEXP".r val IN = "(?i)IN".r + val DIV = "(?i)DIV".r + val BETWEEN = "(?i)BETWEEN".r protected def nodeToExpr(node: Node): Expression = node match { /* Attribute References */ @@ -864,7 +866,7 @@ private[hive] object HiveQl { case Token("-", left :: right:: Nil) => Subtract(nodeToExpr(left), nodeToExpr(right)) case Token("*", left :: right:: Nil) => Multiply(nodeToExpr(left), nodeToExpr(right)) case Token("/", left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right)) - case Token("DIV", left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right)) + case Token(DIV(), left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right)) case Token("%", left :: right:: Nil) => Remainder(nodeToExpr(left), nodeToExpr(right)) /* Comparisons */ @@ -885,7 +887,7 @@ private[hive] object HiveQl { case Token("TOK_FUNCTION", Token(IN(), Nil) :: value :: list) => In(nodeToExpr(value), list.map(nodeToExpr)) case Token("TOK_FUNCTION", - Token("between", Nil) :: + Token(BETWEEN(), Nil) :: Token("KW_FALSE", Nil) :: target :: minValue :: diff --git a/sql/hive/src/test/resources/golden/between-0-570e4c4bf2968368ac2f325c6c940a06 b/sql/hive/src/test/resources/golden/between-0-570e4c4bf2968368ac2f325c6c940a06 new file mode 100644 index 0000000000000..dcd1d8643e3cb --- /dev/null +++ b/sql/hive/src/test/resources/golden/between-0-570e4c4bf2968368ac2f325c6c940a06 @@ -0,0 +1 @@ +2 val_2 diff --git a/sql/hive/src/test/resources/golden/div-0-5e7fc5719c7265bc1d6af52005ebad03 b/sql/hive/src/test/resources/golden/div-0-5e7fc5719c7265bc1d6af52005ebad03 new file mode 100644 index 0000000000000..06b63ea6c2f01 --- /dev/null +++ b/sql/hive/src/test/resources/golden/div-0-5e7fc5719c7265bc1d6af52005ebad03 @@ -0,0 +1 @@ +0 0 0 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 1a2b2f89182ae..125cc18bfb2b5 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 @@ -25,8 +25,10 @@ import org.apache.spark.sql.hive.test.TestHive._ class HiveQuerySuite extends HiveComparisonTest { createQueryTest("between", - "SELECT * FROM src WHERE key between 1 and 2" - ) + "SELECT * FROM src WHERE key Between 1 and 2") + + createQueryTest("div", + "SELECT 1 DIV 2, 1 div 2, 1 dIv 2 FROM src LIMIT 1") test("Query expressed in SQL") { assert(sql("SELECT 1").collect() === Array(Seq(1))) From 41c4a33105c74417192925db355019ba1badeab2 Mon Sep 17 00:00:00 2001 From: witgo Date: Fri, 6 Jun 2014 11:45:21 -0700 Subject: [PATCH 40/54] [SPARK-1841]: update scalatest to version 2.1.5 Author: witgo Closes #713 from witgo/scalatest and squashes the following commits: b627a6a [witgo] merge master 51fb3d6 [witgo] merge master 3771474 [witgo] fix RDDSuite 996d6f9 [witgo] fix TimeStampedWeakValueHashMap test 9dfa4e7 [witgo] merge bug 1479b22 [witgo] merge master 29b9194 [witgo] fix code style 022a7a2 [witgo] fix test dependency a52c0fa [witgo] fix test dependency cd8f59d [witgo] Merge branch 'master' of https://github.com/apache/spark into scalatest 046540d [witgo] fix RDDSuite.scala 2c543b9 [witgo] fix ReplSuite.scala c458928 [witgo] update scalatest to version 2.1.5 --- core/pom.xml | 2 +- .../apache/spark/ContextCleanerSuite.scala | 6 ++--- .../org/apache/spark/ShuffleNettySuite.scala | 4 ++-- .../scala/org/apache/spark/rdd/RDDSuite.scala | 5 +++-- .../spark/scheduler/DAGSchedulerSuite.scala | 4 ++-- .../spark/util/TimeStampedHashMapSuite.scala | 3 ++- pom.xml | 15 +++++++++---- project/SparkBuild.scala | 22 +++++++++---------- .../org/apache/spark/repl/ReplSuite.scala | 6 +++-- .../spark/sql/parquet/ParquetQuerySuite.scala | 4 ++-- .../streaming/BasicOperationsSuite.scala | 12 +++++----- 11 files changed, 47 insertions(+), 36 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 0c746175afa73..c3d6b00a443f1 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -235,7 +235,7 @@ org.easymock - easymock + easymockclassextension test diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 5a8310090890d..dc2db66df60e0 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -25,7 +25,7 @@ import scala.language.postfixOps import scala.util.Random import org.scalatest.{BeforeAndAfter, FunSuite} -import org.scalatest.concurrent.Eventually +import org.scalatest.concurrent.{PatienceConfiguration, Eventually} import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ @@ -76,7 +76,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo tester.assertCleanup() // Verify that shuffles can be re-executed after cleaning up - assert(rdd.collect().toList === collected) + assert(rdd.collect().toList.equals(collected)) } test("cleanup broadcast") { @@ -285,7 +285,7 @@ class CleanerTester( sc.cleaner.get.attachListener(cleanerListener) /** Assert that all the stuff has been cleaned up */ - def assertCleanup()(implicit waitTimeout: Eventually.Timeout) { + def assertCleanup()(implicit waitTimeout: PatienceConfiguration.Timeout) { try { eventually(waitTimeout, interval(100 millis)) { assert(isAllCleanedUp) diff --git a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala index 29d428aa7dc41..47df00050c1e2 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala @@ -23,11 +23,11 @@ class ShuffleNettySuite extends ShuffleSuite with BeforeAndAfterAll { // This test suite should run all tests in ShuffleSuite with Netty shuffle mode. - override def beforeAll(configMap: Map[String, Any]) { + override def beforeAll() { System.setProperty("spark.shuffle.use.netty", "true") } - override def afterAll(configMap: Map[String, Any]) { + override def afterAll() { System.setProperty("spark.shuffle.use.netty", "false") } } 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 286e221e33b78..55af1666df662 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -266,8 +266,9 @@ class RDDSuite extends FunSuite with SharedSparkContext { // we can optionally shuffle to keep the upstream parallel val coalesced5 = data.coalesce(1, shuffle = true) - assert(coalesced5.dependencies.head.rdd.dependencies.head.rdd.asInstanceOf[ShuffledRDD[_, _, _]] != - null) + val isEquals = coalesced5.dependencies.head.rdd.dependencies.head.rdd. + asInstanceOf[ShuffledRDD[_, _, _]] != null + assert(isEquals) // when shuffling, we can increase the number of partitions val coalesced6 = data.coalesce(20, shuffle = true) 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 81e64c1846ed5..7506d56d7e26d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -23,7 +23,7 @@ import scala.language.reflectiveCalls import akka.actor._ import akka.testkit.{ImplicitSender, TestKit, TestActorRef} -import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.{BeforeAndAfter, FunSuiteLike} import org.apache.spark._ import org.apache.spark.rdd.RDD @@ -37,7 +37,7 @@ class BuggyDAGEventProcessActor extends Actor { } } -class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with FunSuite +class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with FunSuiteLike with ImplicitSender with BeforeAndAfter with LocalSparkContext { val conf = new SparkConf diff --git a/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala index 6a5653ed2fb54..c1c605cdb487b 100644 --- a/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala @@ -105,7 +105,8 @@ class TimeStampedHashMapSuite extends FunSuite { map("k1") = strongRef map("k2") = "v2" map("k3") = "v3" - assert(map("k1") === strongRef) + val isEquals = map("k1") == strongRef + assert(isEquals) // clear strong reference to "k1" strongRef = null diff --git a/pom.xml b/pom.xml index 87c8e29ad1069..891468b21bfff 100644 --- a/pom.xml +++ b/pom.xml @@ -459,25 +459,31 @@ org.scalatest scalatest_${scala.binary.version} - 1.9.1 + 2.1.5 test org.easymock - easymock + easymockclassextension 3.1 test org.mockito mockito-all - 1.8.5 + 1.9.0 test org.scalacheck scalacheck_${scala.binary.version} - 1.10.0 + 1.11.3 + test + + + junit + junit + 4.10 test @@ -779,6 +785,7 @@ -unchecked -deprecation -feature + -language:postfixOps -Xms1024m diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d0049a8ac43aa..069913dbaac56 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -270,16 +270,17 @@ object SparkBuild extends Build { */ libraryDependencies ++= Seq( - "io.netty" % "netty-all" % "4.0.17.Final", - "org.eclipse.jetty" % "jetty-server" % jettyVersion, - "org.eclipse.jetty" % "jetty-util" % jettyVersion, - "org.eclipse.jetty" % "jetty-plus" % jettyVersion, - "org.eclipse.jetty" % "jetty-security" % jettyVersion, - "org.scalatest" %% "scalatest" % "1.9.1" % "test", - "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", - "com.novocode" % "junit-interface" % "0.10" % "test", - "org.easymock" % "easymock" % "3.1" % "test", - "org.mockito" % "mockito-all" % "1.8.5" % "test" + "io.netty" % "netty-all" % "4.0.17.Final", + "org.eclipse.jetty" % "jetty-server" % jettyVersion, + "org.eclipse.jetty" % "jetty-util" % jettyVersion, + "org.eclipse.jetty" % "jetty-plus" % jettyVersion, + "org.eclipse.jetty" % "jetty-security" % jettyVersion, + "org.scalatest" %% "scalatest" % "2.1.5" % "test", + "org.scalacheck" %% "scalacheck" % "1.11.3" % "test", + "com.novocode" % "junit-interface" % "0.10" % "test", + "org.easymock" % "easymockclassextension" % "3.1" % "test", + "org.mockito" % "mockito-all" % "1.9.0" % "test", + "junit" % "junit" % "4.10" % "test" ), testOptions += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), @@ -478,7 +479,6 @@ object SparkBuild extends Build { // this non-deterministically. TODO: FIX THIS. parallelExecution in Test := false, libraryDependencies ++= Seq( - "org.scalatest" %% "scalatest" % "1.9.1" % "test", "com.typesafe" %% "scalalogging-slf4j" % "1.0.1" ) ) 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 7c765edd55027..f4ba8d9cc079b 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -67,12 +67,14 @@ class ReplSuite extends FunSuite { } def assertContains(message: String, output: String) { - assert(output.contains(message), + val isContain = output.contains(message) + assert(isContain, "Interpreter output did not contain '" + message + "':\n" + output) } def assertDoesNotContain(message: String, output: String) { - assert(!output.contains(message), + val isContain = output.contains(message) + assert(!isContain, "Interpreter output contained '" + message + "':\n" + output) } 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 b973ceba5fec0..9810520bb9ae6 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,7 +17,7 @@ package org.apache.spark.sql.parquet -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.hadoop.mapreduce.Job @@ -56,7 +56,7 @@ case class OptionalReflectData( doubleField: Option[Double], booleanField: Option[Boolean]) -class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { +class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { import TestData._ TestData // Load test data tables. 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 04925886c39e4..ff6d86c8f81ac 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -92,9 +92,9 @@ class BasicOperationsSuite extends TestSuiteBase { assert(second.size === 5) assert(third.size === 5) - assert(first.flatten.toSet === (1 to 100).toSet) - assert(second.flatten.toSet === (101 to 200).toSet) - assert(third.flatten.toSet === (201 to 300).toSet) + assert(first.flatten.toSet.equals((1 to 100).toSet) ) + assert(second.flatten.toSet.equals((101 to 200).toSet)) + assert(third.flatten.toSet.equals((201 to 300).toSet)) } test("repartition (fewer partitions)") { @@ -111,9 +111,9 @@ class BasicOperationsSuite extends TestSuiteBase { assert(second.size === 2) assert(third.size === 2) - assert(first.flatten.toSet === (1 to 100).toSet) - assert(second.flatten.toSet === (101 to 200).toSet) - assert(third.flatten.toSet === (201 to 300).toSet) + assert(first.flatten.toSet.equals((1 to 100).toSet)) + assert(second.flatten.toSet.equals( (101 to 200).toSet)) + assert(third.flatten.toSet.equals((201 to 300).toSet)) } test("groupByKey") { From a6c72ab16e7a3027739ab419819f5222e270838e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 7 Jun 2014 14:20:33 -0700 Subject: [PATCH 41/54] [SPARK-1994][SQL] Weird data corruption bug when running Spark SQL on data in HDFS Basically there is a race condition (possibly a scala bug?) when these values are recomputed on all of the slaves that results in an incorrect projection being generated (possibly because the GUID uniqueness contract is broken?). In general we should probably enforce that all expression planing occurs on the driver, as is now occurring here. Author: Michael Armbrust Closes #1004 from marmbrus/fixAggBug and squashes the following commits: e0c116c [Michael Armbrust] Compute aggregate expression during planning instead of lazily on workers. --- .../apache/spark/sql/execution/Aggregate.scala | 15 +++++---------- 1 file changed, 5 insertions(+), 10 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 604914e547790..34d88fe4bd7de 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 @@ -77,8 +77,7 @@ case class Aggregate( resultAttribute: AttributeReference) /** A list of aggregates that need to be computed for each group. */ - @transient - private[this] lazy val computedAggregates = aggregateExpressions.flatMap { agg => + private[this] val computedAggregates = aggregateExpressions.flatMap { agg => agg.collect { case a: AggregateExpression => ComputedAggregate( @@ -89,8 +88,7 @@ case class Aggregate( }.toArray /** The schema of the result of all aggregate evaluations */ - @transient - private[this] lazy val computedSchema = computedAggregates.map(_.resultAttribute) + private[this] val computedSchema = computedAggregates.map(_.resultAttribute) /** Creates a new aggregate buffer for a group. */ private[this] def newAggregateBuffer(): Array[AggregateFunction] = { @@ -104,8 +102,7 @@ case class Aggregate( } /** Named attributes used to substitute grouping attributes into the final result. */ - @transient - private[this] lazy val namedGroups = groupingExpressions.map { + private[this] val namedGroups = groupingExpressions.map { case ne: NamedExpression => ne -> ne.toAttribute case e => e -> Alias(e, s"groupingExpr:$e")().toAttribute } @@ -114,16 +111,14 @@ case class Aggregate( * A map of substitutions that are used to insert the aggregate expressions and grouping * expression into the final result expression. */ - @transient - private[this] lazy val resultMap = + private[this] 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 - private[this] lazy val resultExpressions = aggregateExpressions.map { agg => + private[this] val resultExpressions = aggregateExpressions.map { agg => agg.transform { case e: Expression if resultMap.contains(e) => resultMap(e) } From 3ace10dc91e72ebe5013d5106eb0968a77c99d8d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 7 Jun 2014 16:16:37 -0700 Subject: [PATCH 42/54] HOTFIX: Support empty body in merge script Discovered in #992 Author: Patrick Wendell Closes #1007 from pwendell/hotfix and squashes the following commits: af90aa0 [Patrick Wendell] HOTFIX: Support empty body in merge script --- dev/merge_spark_pr.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index e3ac32ef1a12e..ffb70096d6014 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -128,8 +128,9 @@ def merge_pr(pr_num, target_ref): merge_message_flags = [] - for p in [title, body]: - merge_message_flags += ["-m", p] + merge_message_flags += ["-m", title] + if body != None: + merge_message_flags += ["-m", body] authors = "\n".join(["Author: %s" % a for a in distinct_authors]) From 7b877b27053bfb7092e250e01a3b887e1b50a109 Mon Sep 17 00:00:00 2001 From: Neville Li Date: Sat, 7 Jun 2014 16:22:26 -0700 Subject: [PATCH 43/54] SPARK-2056 Set RDD name to input path Author: Neville Li Closes #992 from nevillelyh/master and squashes the following commits: 3011739 [Neville Li] [SPARK-2056] Set RDD name to input path --- core/src/main/scala/org/apache/spark/SparkContext.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d941aea9d7eb2..d721aba709600 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -455,7 +455,7 @@ class SparkContext(config: SparkConf) extends Logging { */ def textFile(path: String, minPartitions: Int = defaultMinPartitions): RDD[String] = { hadoopFile(path, classOf[TextInputFormat], classOf[LongWritable], classOf[Text], - minPartitions).map(pair => pair._2.toString) + minPartitions).map(pair => pair._2.toString).setName(path) } /** @@ -496,7 +496,7 @@ class SparkContext(config: SparkConf) extends Logging { classOf[String], classOf[String], updateConf, - minPartitions) + minPartitions).setName(path) } /** @@ -551,7 +551,7 @@ class SparkContext(config: SparkConf) extends Logging { inputFormatClass, keyClass, valueClass, - minPartitions) + minPartitions).setName(path) } /** @@ -623,7 +623,7 @@ class SparkContext(config: SparkConf) extends Logging { val job = new NewHadoopJob(conf) NewFileInputFormat.addInputPath(job, new Path(path)) val updatedConf = job.getConfiguration - new NewHadoopRDD(this, fClass, kClass, vClass, updatedConf) + new NewHadoopRDD(this, fClass, kClass, vClass, updatedConf).setName(path) } /** From a338834f90556d78119b37985b603ecee85f97ed Mon Sep 17 00:00:00 2001 From: Bernardo Gomez Palacio Date: Sun, 8 Jun 2014 01:24:52 -0700 Subject: [PATCH 44/54] SPARK-2026: Maven Hadoop Profiles Should Set The Hadoop Version The Maven Profiles that refer to hadoopX, e.g. `hadoop2.4`, should set the expected `hadoop.version` and `yarn.version`. e.g. ``` hadoop-2.4 2.4.0 ${hadoop.version} 2.5.0 0.9.0 ``` Builds can still define the `-Dhadoop.version` option but this will correctly default the Hadoop Version to the one that is expected according the profile that is selected. e.g. ```$ mvn -P hadoop-2.4,yarn clean install``` or ```$ mvn -P hadoop-0.23,yarn clean install``` [ticket] : https://issues.apache.org/jira/browse/SPARK-2026 Author : berngp Reviewer : ? Author: Bernardo Gomez Palacio Closes #998 from berngp/feature/SPARK-2026 and squashes the following commits: 07ba4f7 [Bernardo Gomez Palacio] SPARK-2026: Maven Hadoop Profiles Should Set The Hadoop Version --- pom.xml | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 891468b21bfff..0d46bb4114f73 100644 --- a/pom.xml +++ b/pom.xml @@ -209,14 +209,14 @@ spring-releases Spring Release Repository - http://repo.spring.io/libs-release + http://repo.spring.io/libs-release true false - + @@ -987,11 +987,15 @@ avro + + 0.23.10 + hadoop-2.2 + 2.2.0 2.5.0 @@ -999,6 +1003,7 @@ hadoop-2.3 + 2.3.0 2.5.0 0.9.0 @@ -1007,6 +1012,7 @@ hadoop-2.4 + 2.4.0 2.5.0 0.9.0 From ee96e9406613e621837360b15c34ea7c7220a7a3 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Sun, 8 Jun 2014 12:27:34 -0700 Subject: [PATCH 45/54] SPARK-1898: In deploy.yarn.Client, use YarnClient not YarnClientImpl https://issues.apache.org/jira/browse/SPARK-1898 Author: Colin Patrick McCabe Closes #850 from cmccabe/master and squashes the following commits: d66eddc [Colin Patrick McCabe] SPARK-1898: In deploy.yarn.Client, use YarnClient rather than YarnClientImpl --- .../cluster/YarnClientSchedulerBackend.scala | 2 +- .../org/apache/spark/deploy/yarn/Client.scala | 25 ++++++++++++------- 2 files changed, 17 insertions(+), 10 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 e01ed5a57d697..039cf4f276119 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 @@ -112,7 +112,7 @@ private[spark] class YarnClientSchedulerBackend( override def stop() { super.stop() - client.stop() + client.stop logInfo("Stopped") } 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 1b6bfb42a5c1c..393edd1f2d670 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 @@ -25,7 +25,7 @@ import org.apache.hadoop.yarn.api._ 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.client.api.impl.YarnClientImpl +import org.apache.hadoop.yarn.client.api.YarnClient import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, Records} @@ -37,7 +37,9 @@ import org.apache.spark.{Logging, SparkConf} * 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 { + extends ClientBase with Logging { + + val yarnClient = YarnClient.createYarnClient def this(clientArgs: ClientArguments, spConf: SparkConf) = this(clientArgs, new Configuration(), spConf) @@ -53,8 +55,8 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa def runApp(): ApplicationId = { validateArgs() // Initialize and start the client service. - init(yarnConf) - start() + yarnClient.init(yarnConf) + yarnClient.start() // Log details about this YARN cluster (e.g, the number of slave machines/NodeManagers). logClusterResourceDetails() @@ -63,7 +65,7 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa // interface). // Get a new client application. - val newApp = super.createApplication() + val newApp = yarnClient.createApplication() val newAppResponse = newApp.getNewApplicationResponse() val appId = newAppResponse.getApplicationId() @@ -99,11 +101,11 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa } def logClusterResourceDetails() { - val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics + val clusterMetrics: YarnClusterMetrics = yarnClient.getYarnClusterMetrics logInfo("Got Cluster metric info from ApplicationsManager (ASM), number of NodeManagers: " + clusterMetrics.getNumNodeManagers) - val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue) + val queueInfo: QueueInfo = yarnClient.getQueueInfo(args.amQueue) logInfo( """Queue info ... queueName: %s, queueCurrentCapacity: %s, queueMaxCapacity: %s, queueApplicationCount = %s, queueChildQueueCount = %s""".format( queueInfo.getQueueName, @@ -132,15 +134,20 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa def submitApp(appContext: ApplicationSubmissionContext) = { // Submit the application to the applications manager. logInfo("Submitting application to ASM") - super.submitApplication(appContext) + yarnClient.submitApplication(appContext) } + def getApplicationReport(appId: ApplicationId) = + yarnClient.getApplicationReport(appId) + + def stop = yarnClient.stop + def monitorApplication(appId: ApplicationId): Boolean = { val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) while (true) { Thread.sleep(interval) - val report = super.getApplicationReport(appId) + val report = yarnClient.getApplicationReport(appId) logInfo("Application report from ASM: \n" + "\t application identifier: " + appId.toString() + "\n" + From a71c6d1cf0bbc027f633a65042191cf2201330d6 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sun, 8 Jun 2014 14:18:52 -0700 Subject: [PATCH 46/54] SPARK-1628: Add missing hashCode methods in Partitioner subclasses JIRA: https://issues.apache.org/jira/browse/SPARK-1628 Added `hashCode` in HashPartitioner, RangePartitioner, PythonPartitioner and PageRankUtils.CustomPartitioner. Author: zsxwing Closes #549 from zsxwing/SPARK-1628 and squashes the following commits: 2620936 [zsxwing] SPARK-1628: Add missing hashCode methods in Partitioner subclasses --- .../scala/org/apache/spark/Partitioner.scala | 17 ++++++++++++++++- .../spark/api/python/PythonPartitioner.scala | 2 ++ .../spark/examples/bagel/PageRankUtils.scala | 2 ++ 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 9155159cf6aeb..01e918fabec67 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -83,6 +83,8 @@ class HashPartitioner(partitions: Int) extends Partitioner { case _ => false } + + override def hashCode: Int = numPartitions } /** @@ -119,7 +121,7 @@ class RangePartitioner[K : Ordering : ClassTag, V]( } } - def numPartitions = partitions + def numPartitions = rangeBounds.length + 1 private val binarySearch: ((Array[K], K) => Int) = CollectionsUtils.makeBinarySearch[K] @@ -155,4 +157,17 @@ class RangePartitioner[K : Ordering : ClassTag, V]( case _ => false } + + + override def hashCode(): Int = { + val prime = 31 + var result = 1 + var i = 0 + while (i < rangeBounds.length) { + result = prime * result + rangeBounds(i).hashCode + i += 1 + } + result = prime * result + ascending.hashCode + result + } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala index 95bec5030bfdd..e230d222b8604 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala @@ -50,4 +50,6 @@ private[spark] class PythonPartitioner( case _ => false } + + override def hashCode: Int = 31 * numPartitions + pyPartitionFunctionId.hashCode } diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala index b97cb8fb02823..e06f4dcd54442 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala @@ -124,4 +124,6 @@ class CustomPartitioner(partitions: Int) extends Partitioner { c.numPartitions == numPartitions case _ => false } + + override def hashCode: Int = numPartitions } From e9261d0866a610eab29fa332726186b534d1018f Mon Sep 17 00:00:00 2001 From: maji2014 Date: Sun, 8 Jun 2014 15:14:27 -0700 Subject: [PATCH 47/54] Update run-example Old code can only be ran under spark_home and use "bin/run-example". Error "./run-example: line 55: ./bin/spark-submit: No such file or directory" appears when running in other place. So change this Author: maji2014 Closes #1011 from maji2014/master and squashes the following commits: 2cc1af6 [maji2014] Update run-example Closes #988. --- bin/run-example | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/run-example b/bin/run-example index 7caab31daef39..e7a5fe3914fbd 100755 --- a/bin/run-example +++ b/bin/run-example @@ -51,7 +51,7 @@ if [[ ! $EXAMPLE_CLASS == org.apache.spark.examples* ]]; then EXAMPLE_CLASS="org.apache.spark.examples.$EXAMPLE_CLASS" fi -./bin/spark-submit \ +"$FWDIR"/bin/spark-submit \ --master $EXAMPLE_MASTER \ --class $EXAMPLE_CLASS \ "$SPARK_EXAMPLES_JAR" \ From 219dc00b30c8d9c4c0a6ce5d566497a93f21cb57 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 8 Jun 2014 18:39:57 -0700 Subject: [PATCH 48/54] SPARK-1628 follow up: Improve RangePartitioner's documentation. Adding a paragraph clarifying a weird behavior in RangePartitioner. See also #549. Author: Reynold Xin Closes #1012 from rxin/partitioner-doc and squashes the following commits: 6f0109e [Reynold Xin] SPARK-1628 follow up: Improve RangePartitioner's documentation. --- core/src/main/scala/org/apache/spark/Partitioner.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 01e918fabec67..e7f75481939a8 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -90,6 +90,10 @@ class HashPartitioner(partitions: Int) extends Partitioner { /** * A [[org.apache.spark.Partitioner]] that partitions sortable records by range into roughly * equal ranges. The ranges are determined by sampling the content of the RDD passed in. + * + * Note that the actual number of partitions created by the RangePartitioner might not be the same + * as the `partitions` parameter, in the case where the number of sampled records is less than + * the value of `partitions`. */ class RangePartitioner[K : Ordering : ClassTag, V]( partitions: Int, @@ -158,7 +162,6 @@ class RangePartitioner[K : Ordering : ClassTag, V]( false } - override def hashCode(): Int = { val prime = 31 var result = 1 From 15ddbef414d5fd6d4672936ba3c747b5fb7ab52b Mon Sep 17 00:00:00 2001 From: Neville Li Date: Sun, 8 Jun 2014 23:18:27 -0700 Subject: [PATCH 49/54] [SPARK-2067] use relative path for Spark logo in UI Author: Neville Li Closes #1006 from nevillelyh/gh/SPARK-2067 and squashes the following commits: 9ee64cf [Neville Li] [SPARK-2067] use relative path for Spark logo in UI --- core/src/main/scala/org/apache/spark/ui/UIUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 a43314f48112f..1b104253d545d 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -168,7 +168,7 @@ private[spark] object UIUtils extends Logging {