From b3abf0b8d9bca13840eb759953d76905c2ba9b8a Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Wed, 20 May 2015 10:41:18 +0100 Subject: [PATCH 01/17] [SPARK-7663] [MLLIB] Add requirement for word2vec model JIRA issue [link](https://issues.apache.org/jira/browse/SPARK-7663). We should check the model size of word2vec, to prevent the unexpected empty. CC srowen. Author: Xusen Yin Closes #6228 from yinxusen/SPARK-7663 and squashes the following commits: 21770c5 [Xusen Yin] check the vocab size 54ae63e [Xusen Yin] add requirement for word2vec model --- .../main/scala/org/apache/spark/mllib/feature/Word2Vec.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index 731f7576c2335..f65f78299d182 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -158,6 +158,9 @@ class Word2Vec extends Serializable with Logging { .sortWith((a, b) => a.cn > b.cn) vocabSize = vocab.length + require(vocabSize > 0, "The vocabulary size should be > 0. You may need to check " + + "the setting of minCount, which could be large enough to remove all your words in sentences.") + var a = 0 while (a < vocabSize) { vocabHash += vocab(a).word -> a From 09265ad7c85c6de6b568ec329daad632d4a79fa3 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 20 May 2015 19:09:47 +0800 Subject: [PATCH 02/17] [SPARK-7320] [SQL] Add Cube / Rollup for dataframe Add `cube` & `rollup` for DataFrame For example: ```scala testData.rollup($"a" + $"b", $"b").agg(sum($"a" - $"b")) testData.cube($"a" + $"b", $"b").agg(sum($"a" - $"b")) ``` Author: Cheng Hao Closes #6257 from chenghao-intel/rollup and squashes the following commits: 7302319 [Cheng Hao] cancel the implicit keyword a66e38f [Cheng Hao] remove the unnecessary code changes a2869d4 [Cheng Hao] update the code as comments c441777 [Cheng Hao] update the code as suggested 84c9564 [Cheng Hao] Remove the CubedData & RollupedData 279584c [Cheng Hao] hiden the CubedData & RollupedData ef357e1 [Cheng Hao] Add Cube / Rollup for dataframe --- .../org/apache/spark/sql/DataFrame.scala | 104 +++++++++++++++++- .../org/apache/spark/sql/GroupedData.scala | 92 +++++++++++----- .../hive/HiveDataFrameAnalyticsSuite.scala | 62 +++++++++++ 3 files changed, 230 insertions(+), 28 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index adad85806d1ea..d78b4c2f8909c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -685,7 +685,53 @@ class DataFrame private[sql]( * @since 1.3.0 */ @scala.annotation.varargs - def groupBy(cols: Column*): GroupedData = new GroupedData(this, cols.map(_.expr)) + def groupBy(cols: Column*): GroupedData = { + GroupedData(this, cols.map(_.expr), GroupedData.GroupByType) + } + + /** + * Create a multi-dimensional rollup for the current [[DataFrame]] using the specified columns, + * so we can run aggregation on them. + * See [[GroupedData]] for all the available aggregate functions. + * + * {{{ + * // Compute the average for all numeric columns rolluped by department and group. + * df.rollup($"department", $"group").avg() + * + * // Compute the max age and average salary, rolluped by department and gender. + * df.rollup($"department", $"gender").agg(Map( + * "salary" -> "avg", + * "age" -> "max" + * )) + * }}} + * @group dfops + * @since 1.4.0 + */ + @scala.annotation.varargs + def rollup(cols: Column*): GroupedData = { + GroupedData(this, cols.map(_.expr), GroupedData.RollupType) + } + + /** + * Create a multi-dimensional cube for the current [[DataFrame]] using the specified columns, + * so we can run aggregation on them. + * See [[GroupedData]] for all the available aggregate functions. + * + * {{{ + * // Compute the average for all numeric columns cubed by department and group. + * df.cube($"department", $"group").avg() + * + * // Compute the max age and average salary, cubed by department and gender. + * df.cube($"department", $"gender").agg(Map( + * "salary" -> "avg", + * "age" -> "max" + * )) + * }}} + * @group dfops + * @since 1.4.0 + */ + @scala.annotation.varargs + def cube(cols: Column*): GroupedData = GroupedData(this, cols.map(_.expr), GroupedData.CubeType) /** * Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them. @@ -710,7 +756,61 @@ class DataFrame private[sql]( @scala.annotation.varargs def groupBy(col1: String, cols: String*): GroupedData = { val colNames: Seq[String] = col1 +: cols - new GroupedData(this, colNames.map(colName => resolve(colName))) + GroupedData(this, colNames.map(colName => resolve(colName)), GroupedData.GroupByType) + } + + /** + * Create a multi-dimensional rollup for the current [[DataFrame]] using the specified columns, + * so we can run aggregation on them. + * See [[GroupedData]] for all the available aggregate functions. + * + * This is a variant of rollup that can only group by existing columns using column names + * (i.e. cannot construct expressions). + * + * {{{ + * // Compute the average for all numeric columns rolluped by department and group. + * df.rollup("department", "group").avg() + * + * // Compute the max age and average salary, rolluped by department and gender. + * df.rollup($"department", $"gender").agg(Map( + * "salary" -> "avg", + * "age" -> "max" + * )) + * }}} + * @group dfops + * @since 1.4.0 + */ + @scala.annotation.varargs + def rollup(col1: String, cols: String*): GroupedData = { + val colNames: Seq[String] = col1 +: cols + GroupedData(this, colNames.map(colName => resolve(colName)), GroupedData.RollupType) + } + + /** + * Create a multi-dimensional cube for the current [[DataFrame]] using the specified columns, + * so we can run aggregation on them. + * See [[GroupedData]] for all the available aggregate functions. + * + * This is a variant of cube that can only group by existing columns using column names + * (i.e. cannot construct expressions). + * + * {{{ + * // Compute the average for all numeric columns cubed by department and group. + * df.cube("department", "group").avg() + * + * // Compute the max age and average salary, cubed by department and gender. + * df.cube($"department", $"gender").agg(Map( + * "salary" -> "avg", + * "age" -> "max" + * )) + * }}} + * @group dfops + * @since 1.4.0 + */ + @scala.annotation.varargs + def cube(col1: String, cols: String*): GroupedData = { + val colNames: Seq[String] = col1 +: cols + GroupedData(this, colNames.map(colName => resolve(colName)), GroupedData.CubeType) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala index 1381b9f1a6080..f730e4ae00e2b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala @@ -23,9 +23,40 @@ import scala.language.implicitConversions import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.analysis.Star import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.Aggregate +import org.apache.spark.sql.catalyst.plans.logical.{Rollup, Cube, Aggregate} import org.apache.spark.sql.types.NumericType +/** + * Companion object for GroupedData + */ +private[sql] object GroupedData { + def apply( + df: DataFrame, + groupingExprs: Seq[Expression], + groupType: GroupType): GroupedData = { + new GroupedData(df, groupingExprs, groupType: GroupType) + } + + /** + * The Grouping Type + */ + trait GroupType + + /** + * To indicate it's the GroupBy + */ + object GroupByType extends GroupType + + /** + * To indicate it's the CUBE + */ + object CubeType extends GroupType + + /** + * To indicate it's the ROLLUP + */ + object RollupType extends GroupType +} /** * :: Experimental :: @@ -34,19 +65,37 @@ import org.apache.spark.sql.types.NumericType * @since 1.3.0 */ @Experimental -class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) { +class GroupedData protected[sql]( + df: DataFrame, + groupingExprs: Seq[Expression], + private val groupType: GroupedData.GroupType) { - private[sql] implicit def toDF(aggExprs: Seq[NamedExpression]): DataFrame = { - val namedGroupingExprs = groupingExprs.map { - case expr: NamedExpression => expr - case expr: Expression => Alias(expr, expr.prettyString)() + private[this] def toDF(aggExprs: Seq[NamedExpression]): DataFrame = { + val aggregates = if (df.sqlContext.conf.dataFrameRetainGroupColumns) { + val retainedExprs = groupingExprs.map { + case expr: NamedExpression => expr + case expr: Expression => Alias(expr, expr.prettyString)() + } + retainedExprs ++ aggExprs + } else { + aggExprs + } + + groupType match { + case GroupedData.GroupByType => + DataFrame( + df.sqlContext, Aggregate(groupingExprs, aggregates, df.logicalPlan)) + case GroupedData.RollupType => + DataFrame( + df.sqlContext, Rollup(groupingExprs, df.logicalPlan, aggregates)) + case GroupedData.CubeType => + DataFrame( + df.sqlContext, Cube(groupingExprs, df.logicalPlan, aggregates)) } - DataFrame( - df.sqlContext, Aggregate(groupingExprs, namedGroupingExprs ++ aggExprs, df.logicalPlan)) } private[this] def aggregateNumericColumns(colNames: String*)(f: Expression => Expression) - : Seq[NamedExpression] = { + : DataFrame = { val columnExprs = if (colNames.isEmpty) { // No columns specified. Use all numeric columns. @@ -63,10 +112,10 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) namedExpr } } - columnExprs.map { c => + toDF(columnExprs.map { c => val a = f(c) Alias(a, a.prettyString)() - } + }) } private[this] def strToExpr(expr: String): (Expression => Expression) = { @@ -119,10 +168,10 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) * @since 1.3.0 */ def agg(exprs: Map[String, String]): DataFrame = { - exprs.map { case (colName, expr) => + toDF(exprs.map { case (colName, expr) => val a = strToExpr(expr)(df(colName).expr) Alias(a, a.prettyString)() - }.toSeq + }.toSeq) } /** @@ -175,19 +224,10 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) */ @scala.annotation.varargs def agg(expr: Column, exprs: Column*): DataFrame = { - val aggExprs = (expr +: exprs).map(_.expr).map { + toDF((expr +: exprs).map(_.expr).map { case expr: NamedExpression => expr case expr: Expression => Alias(expr, expr.prettyString)() - } - if (df.sqlContext.conf.dataFrameRetainGroupColumns) { - val retainedExprs = groupingExprs.map { - case expr: NamedExpression => expr - case expr: Expression => Alias(expr, expr.prettyString)() - } - DataFrame(df.sqlContext, Aggregate(groupingExprs, retainedExprs ++ aggExprs, df.logicalPlan)) - } else { - DataFrame(df.sqlContext, Aggregate(groupingExprs, aggExprs, df.logicalPlan)) - } + }) } /** @@ -196,7 +236,7 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) * * @since 1.3.0 */ - def count(): DataFrame = Seq(Alias(Count(Literal(1)), "count")()) + def count(): DataFrame = toDF(Seq(Alias(Count(Literal(1)), "count")())) /** * Compute the average value for each numeric columns for each group. This is an alias for `avg`. @@ -256,5 +296,5 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) @scala.annotation.varargs def sum(colNames: String*): DataFrame = { aggregateNumericColumns(colNames:_*)(Sum) - } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala new file mode 100644 index 0000000000000..3ad05f482504c --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.hive.test.TestHive.implicits._ + +case class TestData2Int(a: Int, b: Int) + +// TODO ideally we should put the test suite into the package `sql`, as +// `hive` package is optional in compiling, however, `SQLContext.sql` doesn't +// support the `cube` or `rollup` yet. +class HiveDataFrameAnalyticsSuite extends QueryTest { + val testData = + TestHive.sparkContext.parallelize( + TestData2Int(1, 2) :: + TestData2Int(2, 4) :: Nil).toDF() + + testData.registerTempTable("mytable") + + test("rollup") { + checkAnswer( + testData.rollup($"a" + $"b", $"b").agg(sum($"a" - $"b")), + sql("select a + b, b, sum(a - b) from mytable group by a + b, b with rollup").collect() + ) + + checkAnswer( + testData.rollup("a", "b").agg(sum("b")), + sql("select a, b, sum(b) from mytable group by a, b with rollup").collect() + ) + } + + test("cube") { + checkAnswer( + testData.cube($"a" + $"b", $"b").agg(sum($"a" - $"b")), + sql("select a + b, b, sum(a - b) from mytable group by a + b, b with cube").collect() + ) + + checkAnswer( + testData.cube("a", "b").agg(sum("b")), + sql("select a, b, sum(b) from mytable group by a, b with cube").collect() + ) + } +} From 3ddf051ee7256f642f8a17768d161c7b5f55c7e1 Mon Sep 17 00:00:00 2001 From: ehnalis Date: Wed, 20 May 2015 08:27:39 -0500 Subject: [PATCH 03/17] [SPARK-7533] [YARN] Decrease spacing between AM-RM heartbeats. Added faster RM-heartbeats on pending container allocations with multiplicative back-off. Also updated related documentations. Author: ehnalis Closes #6082 from ehnalis/yarn and squashes the following commits: a1d2101 [ehnalis] MIss-spell fixed. 90f8ba4 [ehnalis] Changed default HB values. 6120295 [ehnalis] Removed the bug, when allocation heartbeat would not start from initial value. 08bac63 [ehnalis] Refined style, grammar, removed duplicated code. 073d283 [ehnalis] [SPARK-7533] [YARN] Decrease spacing between AM-RM heartbeats. d4408c9 [ehnalis] [SPARK-7533] [YARN] Decrease spacing between AM-RM heartbeats. --- docs/running-on-yarn.md | 15 +++++++- .../spark/deploy/yarn/ApplicationMaster.scala | 34 ++++++++++++++----- 2 files changed, 39 insertions(+), 10 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 51c1339165024..9d55f435e80ad 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -71,9 +71,22 @@ Most of the configs are the same for Spark on YARN as for other deployment modes spark.yarn.scheduler.heartbeat.interval-ms - 5000 + 3000 The interval in ms in which the Spark application master heartbeats into the YARN ResourceManager. + The value is capped at half the value of YARN's configuration for the expiry interval + (yarn.am.liveness-monitor.expiry-interval-ms). + + + + spark.yarn.scheduler.initial-allocation.interval + 200ms + + The initial interval in which the Spark application master eagerly heartbeats to the YARN ResourceManager + when there are pending container allocation requests. It should be no larger than + spark.yarn.scheduler.heartbeat.interval-ms. The allocation interval will doubled on + successive eager heartbeats if pending containers still exist, until + spark.yarn.scheduler.heartbeat.interval-ms is reached. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 29752969e6152..63a6f2e9472c1 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -300,11 +300,14 @@ private[spark] class ApplicationMaster( val expiryInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) // we want to be reasonably responsive without causing too many requests to RM. - val schedulerInterval = - sparkConf.getTimeAsMs("spark.yarn.scheduler.heartbeat.interval-ms", "5s") + val heartbeatInterval = math.max(0, math.min(expiryInterval / 2, + sparkConf.getTimeAsMs("spark.yarn.scheduler.heartbeat.interval-ms", "3s"))) - // must be <= expiryInterval / 2. - val interval = math.max(0, math.min(expiryInterval / 2, schedulerInterval)) + // we want to check more frequently for pending containers + val initialAllocationInterval = math.min(heartbeatInterval, + sparkConf.getTimeAsMs("spark.yarn.scheduler.initial-allocation.interval", "200ms")) + + var nextAllocationInterval = initialAllocationInterval // The number of failures in a row until Reporter thread give up val reporterMaxFailures = sparkConf.getInt("spark.yarn.scheduler.reporterThread.maxFailures", 5) @@ -330,15 +333,27 @@ private[spark] class ApplicationMaster( if (!NonFatal(e) || failureCount >= reporterMaxFailures) { finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_REPORTER_FAILURE, "Exception was thrown " + - s"${failureCount} time(s) from Reporter thread.") - + s"$failureCount time(s) from Reporter thread.") } else { - logWarning(s"Reporter thread fails ${failureCount} time(s) in a row.", e) + logWarning(s"Reporter thread fails $failureCount time(s) in a row.", e) } } } try { - Thread.sleep(interval) + val numPendingAllocate = allocator.getNumPendingAllocate + val sleepInterval = + if (numPendingAllocate > 0) { + val currentAllocationInterval = + math.min(heartbeatInterval, nextAllocationInterval) + nextAllocationInterval *= 2 + currentAllocationInterval + } else { + nextAllocationInterval = initialAllocationInterval + heartbeatInterval + } + logDebug(s"Number of pending allocations is $numPendingAllocate. " + + s"Sleeping for $sleepInterval.") + Thread.sleep(sleepInterval) } catch { case e: InterruptedException => } @@ -349,7 +364,8 @@ private[spark] class ApplicationMaster( t.setDaemon(true) t.setName("Reporter") t.start() - logInfo("Started progress reporter thread - sleep time : " + interval) + logInfo(s"Started progress reporter thread with (heartbeat : $heartbeatInterval, " + + s"initial allocation : $initialAllocationInterval) intervals") t } From 589b12f8e62ec5d10713ce057756ebc791e7ddc6 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 20 May 2015 07:46:17 -0700 Subject: [PATCH 04/17] [SPARK-7654] [MLLIB] Migrate MLlib to the DataFrame reader/writer API parquetFile -> read.parquet rxin Author: Xiangrui Meng Closes #6281 from mengxr/SPARK-7654 and squashes the following commits: a79b612 [Xiangrui Meng] parquetFile -> read.parquet --- .../org/apache/spark/mllib/classification/NaiveBayes.scala | 4 ++-- .../mllib/classification/impl/GLMClassificationModel.scala | 2 +- .../apache/spark/mllib/clustering/GaussianMixtureModel.scala | 2 +- .../scala/org/apache/spark/mllib/clustering/KMeansModel.scala | 2 +- .../main/scala/org/apache/spark/mllib/feature/Word2Vec.scala | 2 +- .../spark/mllib/recommendation/MatrixFactorizationModel.scala | 4 ++-- .../apache/spark/mllib/regression/IsotonicRegression.scala | 2 +- .../spark/mllib/regression/impl/GLMRegressionModel.scala | 2 +- .../org/apache/spark/mllib/tree/model/DecisionTreeModel.scala | 2 +- .../apache/spark/mllib/tree/model/treeEnsembleModels.scala | 2 +- 10 files changed, 12 insertions(+), 12 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index 53fb2cba03cbf..cffe9ef1e0b2a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -153,7 +153,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { def load(sc: SparkContext, path: String): NaiveBayesModel = { val sqlContext = new SQLContext(sc) // Load Parquet data. - val dataRDD = sqlContext.parquetFile(dataPath(path)) + val dataRDD = sqlContext.read.parquet(dataPath(path)) // Check schema explicitly since erasure makes it hard to use match-case for checking. checkSchema[Data](dataRDD.schema) val dataArray = dataRDD.select("labels", "pi", "theta", "modelType").take(1) @@ -199,7 +199,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { def load(sc: SparkContext, path: String): NaiveBayesModel = { val sqlContext = new SQLContext(sc) // Load Parquet data. - val dataRDD = sqlContext.parquetFile(dataPath(path)) + val dataRDD = sqlContext.read.parquet(dataPath(path)) // Check schema explicitly since erasure makes it hard to use match-case for checking. checkSchema[Data](dataRDD.schema) val dataArray = dataRDD.select("labels", "pi", "theta").take(1) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala index d842ec57b2f52..fe09f6b75d28b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala @@ -75,7 +75,7 @@ private[classification] object GLMClassificationModel { def loadData(sc: SparkContext, path: String, modelClass: String): Data = { val datapath = Loader.dataPath(path) val sqlContext = new SQLContext(sc) - val dataRDD = sqlContext.parquetFile(datapath) + val dataRDD = sqlContext.read.parquet(datapath) val dataArray = dataRDD.select("weights", "intercept", "threshold").take(1) assert(dataArray.size == 1, s"Unable to load $modelClass data from: $datapath") val data = dataArray(0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index 731b43a1be574..86353aed81156 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -132,7 +132,7 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { def load(sc: SparkContext, path: String): GaussianMixtureModel = { val dataPath = Loader.dataPath(path) val sqlContext = new SQLContext(sc) - val dataFrame = sqlContext.parquetFile(dataPath) + val dataFrame = sqlContext.read.parquet(dataPath) val dataArray = dataFrame.select("weight", "mu", "sigma").collect() // Check schema explicitly since erasure makes it hard to use match-case for checking. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 252e166e85cef..8ecb3df11d95e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -120,7 +120,7 @@ object KMeansModel extends Loader[KMeansModel] { assert(className == thisClassName) assert(formatVersion == thisFormatVersion) val k = (metadata \ "k").extract[Int] - val centriods = sqlContext.parquetFile(Loader.dataPath(path)) + val centriods = sqlContext.read.parquet(Loader.dataPath(path)) Loader.checkSchema[Cluster](centriods.schema) val localCentriods = centriods.map(Cluster.apply).collect() assert(k == localCentriods.size) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index f65f78299d182..9106b73dfcd76 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -559,7 +559,7 @@ object Word2VecModel extends Loader[Word2VecModel] { def load(sc: SparkContext, path: String): Word2VecModel = { val dataPath = Loader.dataPath(path) val sqlContext = new SQLContext(sc) - val dataFrame = sqlContext.parquetFile(dataPath) + val dataFrame = sqlContext.read.parquet(dataPath) val dataArray = dataFrame.select("word", "vector").collect() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index b960fbc5bf5f5..93aa41e49961e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -292,11 +292,11 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { assert(className == thisClassName) assert(formatVersion == thisFormatVersion) val rank = (metadata \ "rank").extract[Int] - val userFeatures = sqlContext.parquetFile(userPath(path)) + val userFeatures = sqlContext.read.parquet(userPath(path)) .map { case Row(id: Int, features: Seq[_]) => (id, features.asInstanceOf[Seq[Double]].toArray) } - val productFeatures = sqlContext.parquetFile(productPath(path)) + val productFeatures = sqlContext.read.parquet(productPath(path)) .map { case Row(id: Int, features: Seq[_]) => (id, features.asInstanceOf[Seq[Double]].toArray) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala index 22b9b22a871f0..3ea63dd8c0acd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala @@ -189,7 +189,7 @@ object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] { def load(sc: SparkContext, path: String): (Array[Double], Array[Double]) = { val sqlContext = new SQLContext(sc) - val dataRDD = sqlContext.parquetFile(dataPath(path)) + val dataRDD = sqlContext.read.parquet(dataPath(path)) checkSchema[Data](dataRDD.schema) val dataArray = dataRDD.select("boundary", "prediction").collect() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala index 2aa0e9ef96d48..317d3a5702636 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala @@ -72,7 +72,7 @@ private[regression] object GLMRegressionModel { def loadData(sc: SparkContext, path: String, modelClass: String, numFeatures: Int): Data = { val datapath = Loader.dataPath(path) val sqlContext = new SQLContext(sc) - val dataRDD = sqlContext.parquetFile(datapath) + val dataRDD = sqlContext.read.parquet(datapath) val dataArray = dataRDD.select("weights", "intercept").take(1) assert(dataArray.size == 1, s"Unable to load $modelClass data from: $datapath") val data = dataArray(0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index a558f84c8d506..25bb1453db404 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -230,7 +230,7 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { val datapath = Loader.dataPath(path) val sqlContext = new SQLContext(sc) // Load Parquet data. - val dataRDD = sqlContext.parquetFile(datapath) + val dataRDD = sqlContext.read.parquet(datapath) // Check schema explicitly since erasure makes it hard to use match-case for checking. Loader.checkSchema[NodeData](dataRDD.schema) val nodes = dataRDD.map(NodeData.apply) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala index f9cd0140fe63f..1e3333d8d81d0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala @@ -437,7 +437,7 @@ private[tree] object TreeEnsembleModel extends Logging { treeAlgo: String): Array[DecisionTreeModel] = { val datapath = Loader.dataPath(path) val sqlContext = new SQLContext(sc) - val nodes = sqlContext.parquetFile(datapath).map(NodeData.apply) + val nodes = sqlContext.read.parquet(datapath).map(NodeData.apply) val trees = constructTrees(nodes) trees.map(new DecisionTreeModel(_, Algo.fromString(treeAlgo))) } From 98a46f9dffec294386f6c39acafa7f11adb87a8f Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 20 May 2015 07:55:51 -0700 Subject: [PATCH 05/17] [SPARK-6094] [MLLIB] Add MultilabelMetrics in PySpark/MLlib Add MultilabelMetrics in PySpark/MLlib Author: Yanbo Liang Closes #6276 from yanboliang/spark-6094 and squashes the following commits: b8e3343 [Yanbo Liang] Add MultilabelMetrics in PySpark/MLlib --- .../mllib/evaluation/MultilabelMetrics.scala | 8 ++ python/pyspark/mllib/evaluation.py | 117 ++++++++++++++++++ 2 files changed, 125 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala index a8378a76d20ae..bf6eb1d5bd2ab 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.evaluation import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ +import org.apache.spark.sql.DataFrame /** * Evaluator for multilabel classification. @@ -27,6 +28,13 @@ import org.apache.spark.SparkContext._ */ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])]) { + /** + * An auxiliary constructor taking a DataFrame. + * @param predictionAndLabels a DataFrame with two double array columns: prediction and label + */ + private[mllib] def this(predictionAndLabels: DataFrame) = + this(predictionAndLabels.map(r => (r.getSeq[Double](0).toArray, r.getSeq[Double](1).toArray))) + private lazy val numDocs: Long = predictionAndLabels.count() private lazy val numLabels: Long = predictionAndLabels.flatMap { case (_, labels) => diff --git a/python/pyspark/mllib/evaluation.py b/python/pyspark/mllib/evaluation.py index a5e5ddc8fe506..aab5e5f4b77b5 100644 --- a/python/pyspark/mllib/evaluation.py +++ b/python/pyspark/mllib/evaluation.py @@ -343,6 +343,123 @@ def ndcgAt(self, k): return self.call("ndcgAt", int(k)) +class MultilabelMetrics(JavaModelWrapper): + """ + Evaluator for multilabel classification. + + >>> predictionAndLabels = sc.parallelize([([0.0, 1.0], [0.0, 2.0]), ([0.0, 2.0], [0.0, 1.0]), + ... ([], [0.0]), ([2.0], [2.0]), ([2.0, 0.0], [2.0, 0.0]), + ... ([0.0, 1.0, 2.0], [0.0, 1.0]), ([1.0], [1.0, 2.0])]) + >>> metrics = MultilabelMetrics(predictionAndLabels) + >>> metrics.precision(0.0) + 1.0 + >>> metrics.recall(1.0) + 0.66... + >>> metrics.f1Measure(2.0) + 0.5 + >>> metrics.precision() + 0.66... + >>> metrics.recall() + 0.64... + >>> metrics.f1Measure() + 0.63... + >>> metrics.microPrecision + 0.72... + >>> metrics.microRecall + 0.66... + >>> metrics.microF1Measure + 0.69... + >>> metrics.hammingLoss + 0.33... + >>> metrics.subsetAccuracy + 0.28... + >>> metrics.accuracy + 0.54... + """ + + def __init__(self, predictionAndLabels): + sc = predictionAndLabels.ctx + sql_ctx = SQLContext(sc) + df = sql_ctx.createDataFrame(predictionAndLabels, + schema=sql_ctx._inferSchema(predictionAndLabels)) + java_class = sc._jvm.org.apache.spark.mllib.evaluation.MultilabelMetrics + java_model = java_class(df._jdf) + super(MultilabelMetrics, self).__init__(java_model) + + def precision(self, label=None): + """ + Returns precision or precision for a given label (category) if specified. + """ + if label is None: + return self.call("precision") + else: + return self.call("precision", float(label)) + + def recall(self, label=None): + """ + Returns recall or recall for a given label (category) if specified. + """ + if label is None: + return self.call("recall") + else: + return self.call("recall", float(label)) + + def f1Measure(self, label=None): + """ + Returns f1Measure or f1Measure for a given label (category) if specified. + """ + if label is None: + return self.call("f1Measure") + else: + return self.call("f1Measure", float(label)) + + @property + def microPrecision(self): + """ + Returns micro-averaged label-based precision. + (equals to micro-averaged document-based precision) + """ + return self.call("microPrecision") + + @property + def microRecall(self): + """ + Returns micro-averaged label-based recall. + (equals to micro-averaged document-based recall) + """ + return self.call("microRecall") + + @property + def microF1Measure(self): + """ + Returns micro-averaged label-based f1-measure. + (equals to micro-averaged document-based f1-measure) + """ + return self.call("microF1Measure") + + @property + def hammingLoss(self): + """ + Returns Hamming-loss. + """ + return self.call("hammingLoss") + + @property + def subsetAccuracy(self): + """ + Returns subset accuracy. + (for equal sets of labels) + """ + return self.call("subsetAccuracy") + + @property + def accuracy(self): + """ + Returns accuracy. + """ + return self.call("accuracy") + + def _test(): import doctest from pyspark import SparkContext From b631bf73b9f288f37c98b806be430b22485880e5 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 20 May 2015 11:23:40 -0700 Subject: [PATCH 06/17] [SPARK-7713] [SQL] Use shared broadcast hadoop conf for partitioned table scan. https://issues.apache.org/jira/browse/SPARK-7713 I tested the performance with the following code: ```scala import sqlContext._ import sqlContext.implicits._ (1 to 5000).foreach { i => val df = (1 to 1000).map(j => (j, s"str$j")).toDF("a", "b").save(s"/tmp/partitioned/i=$i") } sqlContext.sql(""" CREATE TEMPORARY TABLE partitionedParquet USING org.apache.spark.sql.parquet OPTIONS ( path '/tmp/partitioned' )""") table("partitionedParquet").explain(true) ``` In our master `explain` takes 40s in my laptop. With this PR, `explain` takes 14s. Author: Yin Huai Closes #6252 from yhuai/broadcastHadoopConf and squashes the following commits: 6fa73df [Yin Huai] Address comments of Josh and Andrew. 807fbf9 [Yin Huai] Make the new buildScan and SqlNewHadoopRDD private sql. e393555 [Yin Huai] Cheng's comments. 2eb53bb [Yin Huai] Use a shared broadcast Hadoop Configuration for partitioned HadoopFsRelations. --- .../apache/spark/sql/parquet/newParquet.scala | 113 +++++--- .../sql/sources/DataSourceStrategy.scala | 19 +- .../spark/sql/sources/SqlNewHadoopRDD.scala | 268 ++++++++++++++++++ .../apache/spark/sql/sources/interfaces.scala | 35 ++- 4 files changed, 387 insertions(+), 48 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/sources/SqlNewHadoopRDD.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 7ca44f7b81a2d..c35b7eff82af5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConversions._ import scala.util.Try import com.google.common.base.Objects +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ @@ -32,13 +33,14 @@ import parquet.hadoop._ import parquet.hadoop.metadata.CompressionCodecName import parquet.hadoop.util.ContextUtil +import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD._ -import org.apache.spark.rdd.{NewHadoopPartition, NewHadoopRDD, RDD} +import org.apache.spark.rdd.RDD import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.{Row, SQLConf, SQLContext} -import org.apache.spark.{Logging, Partition => SparkPartition, SparkException} +import org.apache.spark.{Partition => SparkPartition, SparkEnv, SerializableWritable, Logging, SparkException} private[sql] class DefaultSource extends HadoopFsRelationProvider { override def createRelation( @@ -233,40 +235,20 @@ private[sql] class ParquetRelation2( override def buildScan( requiredColumns: Array[String], filters: Array[Filter], - inputFiles: Array[FileStatus]): RDD[Row] = { - - val job = new Job(SparkHadoopUtil.get.conf) - val conf = ContextUtil.getConfiguration(job) - - ParquetInputFormat.setReadSupportClass(job, classOf[RowReadSupport]) - - if (inputFiles.nonEmpty) { - FileInputFormat.setInputPaths(job, inputFiles.map(_.getPath): _*) - } - - // Try to push down filters when filter push-down is enabled. - if (sqlContext.conf.parquetFilterPushDown) { - filters - // Collects all converted Parquet filter predicates. Notice that not all predicates can be - // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` - // is used here. - .flatMap(ParquetFilters.createFilter(dataSchema, _)) - .reduceOption(FilterApi.and) - .foreach(ParquetInputFormat.setFilterPredicate(conf, _)) - } - - conf.set(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, { - val requestedSchema = StructType(requiredColumns.map(dataSchema(_))) - ParquetTypesConverter.convertToString(requestedSchema.toAttributes) - }) - - conf.set( - RowWriteSupport.SPARK_ROW_SCHEMA, - ParquetTypesConverter.convertToString(dataSchema.toAttributes)) - - // Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata + inputFiles: Array[FileStatus], + broadcastedConf: Broadcast[SerializableWritable[Configuration]]): RDD[Row] = { val useMetadataCache = sqlContext.getConf(SQLConf.PARQUET_CACHE_METADATA, "true").toBoolean - conf.set(SQLConf.PARQUET_CACHE_METADATA, useMetadataCache.toString) + val parquetFilterPushDown = sqlContext.conf.parquetFilterPushDown + // Create the function to set variable Parquet confs at both driver and executor side. + val initLocalJobFuncOpt = + ParquetRelation2.initializeLocalJobFunc( + requiredColumns, + filters, + dataSchema, + useMetadataCache, + parquetFilterPushDown) _ + // Create the function to set input paths at the driver side. + val setInputPaths = ParquetRelation2.initializeDriverSideJobFunc(inputFiles) _ val footers = inputFiles.map(f => metadataCache.footers(f.getPath)) @@ -274,12 +256,14 @@ private[sql] class ParquetRelation2( // After upgrading to Parquet 1.6.0, we should be able to stop caching `FileStatus` objects and // footers. Especially when a global arbitrative schema (either from metastore or data source // DDL) is available. - new NewHadoopRDD( - sqlContext.sparkContext, - classOf[FilteringParquetRowInputFormat], - classOf[Void], - classOf[Row], - conf) { + new SqlNewHadoopRDD( + sc = sqlContext.sparkContext, + broadcastedConf = broadcastedConf, + initDriverSideJobFuncOpt = Some(setInputPaths), + initLocalJobFuncOpt = Some(initLocalJobFuncOpt), + inputFormatClass = classOf[FilteringParquetRowInputFormat], + keyClass = classOf[Void], + valueClass = classOf[Row]) { val cacheMetadata = useMetadataCache @@ -311,11 +295,11 @@ private[sql] class ParquetRelation2( new FilteringParquetRowInputFormat } - val jobContext = newJobContext(getConf, jobId) + val jobContext = newJobContext(getConf(isDriverSide = true), jobId) val rawSplits = inputFormat.getSplits(jobContext) Array.tabulate[SparkPartition](rawSplits.size) { i => - new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) + new SqlNewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) } } }.values @@ -452,6 +436,49 @@ private[sql] object ParquetRelation2 extends Logging { // internally. private[sql] val METASTORE_SCHEMA = "metastoreSchema" + /** This closure sets various Parquet configurations at both driver side and executor side. */ + private[parquet] def initializeLocalJobFunc( + requiredColumns: Array[String], + filters: Array[Filter], + dataSchema: StructType, + useMetadataCache: Boolean, + parquetFilterPushDown: Boolean)(job: Job): Unit = { + val conf = job.getConfiguration + conf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[RowReadSupport].getName()) + + // Try to push down filters when filter push-down is enabled. + if (parquetFilterPushDown) { + filters + // Collects all converted Parquet filter predicates. Notice that not all predicates can be + // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` + // is used here. + .flatMap(ParquetFilters.createFilter(dataSchema, _)) + .reduceOption(FilterApi.and) + .foreach(ParquetInputFormat.setFilterPredicate(conf, _)) + } + + conf.set(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, { + val requestedSchema = StructType(requiredColumns.map(dataSchema(_))) + ParquetTypesConverter.convertToString(requestedSchema.toAttributes) + }) + + conf.set( + RowWriteSupport.SPARK_ROW_SCHEMA, + ParquetTypesConverter.convertToString(dataSchema.toAttributes)) + + // Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata + conf.set(SQLConf.PARQUET_CACHE_METADATA, useMetadataCache.toString) + } + + /** This closure sets input paths at the driver side. */ + private[parquet] def initializeDriverSideJobFunc( + inputFiles: Array[FileStatus])(job: Job): Unit = { + // We side the input paths at the driver side. + if (inputFiles.nonEmpty) { + FileInputFormat.setInputPaths(job, inputFiles.map(_.getPath): _*) + } + } + private[parquet] def readSchema( footers: Seq[Footer], sqlContext: SQLContext): Option[StructType] = { footers.map { footer => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index 1615a6dcbdb2a..550090d22d551 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.sources -import org.apache.spark.Logging +import org.apache.spark.{SerializableWritable, Logging} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.{RDD, UnionRDD} import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ @@ -84,11 +85,16 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { // Scanning non-partitioned HadoopFsRelation case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: HadoopFsRelation)) => + // See buildPartitionedTableScan for the reason that we need to create a shard + // broadcast HadoopConf. + val sharedHadoopConf = SparkHadoopUtil.get.conf + val confBroadcast = + t.sqlContext.sparkContext.broadcast(new SerializableWritable(sharedHadoopConf)) pruneFilterProject( l, projectList, filters, - (a, f) => t.buildScan(a, f, t.paths)) :: Nil + (a, f) => t.buildScan(a, f, t.paths, confBroadcast)) :: Nil case l @ LogicalRelation(t: TableScan) => createPhysicalRDD(l.relation, l.output, t.buildScan()) :: Nil @@ -115,6 +121,12 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { val output = projections.map(_.toAttribute) val relation = logicalRelation.relation.asInstanceOf[HadoopFsRelation] + // Because we are creating one RDD per partition, we need to have a shared HadoopConf. + // Otherwise, the cost of broadcasting HadoopConf in every RDD will be high. + val sharedHadoopConf = SparkHadoopUtil.get.conf + val confBroadcast = + relation.sqlContext.sparkContext.broadcast(new SerializableWritable(sharedHadoopConf)) + // Builds RDD[Row]s for each selected partition. val perPartitionRows = partitions.map { case Partition(partitionValues, dir) => // The table scan operator (PhysicalRDD) which retrieves required columns from data files. @@ -132,7 +144,8 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { // assuming partition columns data stored in data files are always consistent with those // partition values encoded in partition directory paths. val nonPartitionColumns = requiredColumns.filterNot(partitionColNames.contains) - val dataRows = relation.buildScan(nonPartitionColumns, filters, Array(dir)) + val dataRows = + relation.buildScan(nonPartitionColumns, filters, Array(dir), confBroadcast) // Merges data values with partition values. mergeWithPartitionValues( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/SqlNewHadoopRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/SqlNewHadoopRDD.scala new file mode 100644 index 0000000000000..0c7bb6e50cd98 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/SqlNewHadoopRDD.scala @@ -0,0 +1,268 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources + +import java.text.SimpleDateFormat +import java.util.Date + +import org.apache.hadoop.conf.{Configurable, Configuration} +import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, FileSplit} +import org.apache.spark.broadcast.Broadcast + +import org.apache.spark.{Partition => SparkPartition, _} +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.executor.DataReadMethod +import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil +import org.apache.spark.rdd.{RDD, HadoopRDD} +import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils + +import scala.reflect.ClassTag + +private[spark] class SqlNewHadoopPartition( + rddId: Int, + val index: Int, + @transient rawSplit: InputSplit with Writable) + extends SparkPartition { + + val serializableHadoopSplit = new SerializableWritable(rawSplit) + + override def hashCode(): Int = 41 * (41 + rddId) + index +} + +/** + * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, + * sources in HBase, or S3), using the new MapReduce API (`org.apache.hadoop.mapreduce`). + * It is based on [[org.apache.spark.rdd.NewHadoopRDD]]. It has three additions. + * 1. A shared broadcast Hadoop Configuration. + * 2. An optional closure `initDriverSideJobFuncOpt` that set configurations at the driver side + * to the shared Hadoop Configuration. + * 3. An optional closure `initLocalJobFuncOpt` that set configurations at both the driver side + * and the executor side to the shared Hadoop Configuration. + * + * Note: This is RDD is basically a cloned version of [[org.apache.spark.rdd.NewHadoopRDD]] with + * changes based on [[org.apache.spark.rdd.HadoopRDD]]. In future, this functionality will be + * folded into core. + */ +private[sql] class SqlNewHadoopRDD[K, V]( + @transient sc : SparkContext, + broadcastedConf: Broadcast[SerializableWritable[Configuration]], + @transient initDriverSideJobFuncOpt: Option[Job => Unit], + initLocalJobFuncOpt: Option[Job => Unit], + inputFormatClass: Class[_ <: InputFormat[K, V]], + keyClass: Class[K], + valueClass: Class[V]) + extends RDD[(K, V)](sc, Nil) + with SparkHadoopMapReduceUtil + with Logging { + + if (initLocalJobFuncOpt.isDefined) { + sc.clean(initLocalJobFuncOpt.get) + } + + protected def getJob(): Job = { + val conf: Configuration = broadcastedConf.value.value + // "new Job" will make a copy of the conf. Then, it is + // safe to mutate conf properties with initLocalJobFuncOpt + // and initDriverSideJobFuncOpt. + val newJob = new Job(conf) + initLocalJobFuncOpt.map(f => f(newJob)) + newJob + } + + def getConf(isDriverSide: Boolean): Configuration = { + val job = getJob() + if (isDriverSide) { + initDriverSideJobFuncOpt.map(f => f(job)) + } + job.getConfiguration + } + + private val jobTrackerId: String = { + val formatter = new SimpleDateFormat("yyyyMMddHHmm") + formatter.format(new Date()) + } + + @transient protected val jobId = new JobID(jobTrackerId, id) + + override def getPartitions: Array[SparkPartition] = { + val conf = getConf(isDriverSide = true) + val inputFormat = inputFormatClass.newInstance + inputFormat match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => + } + val jobContext = newJobContext(conf, jobId) + val rawSplits = inputFormat.getSplits(jobContext).toArray + val result = new Array[SparkPartition](rawSplits.size) + for (i <- 0 until rawSplits.size) { + result(i) = + new SqlNewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) + } + result + } + + override def compute( + theSplit: SparkPartition, + context: TaskContext): InterruptibleIterator[(K, V)] = { + val iter = new Iterator[(K, V)] { + val split = theSplit.asInstanceOf[SqlNewHadoopPartition] + logInfo("Input split: " + split.serializableHadoopSplit) + val conf = getConf(isDriverSide = false) + + val inputMetrics = context.taskMetrics + .getInputMetricsForReadMethod(DataReadMethod.Hadoop) + + // Find a function that will return the FileSystem bytes read by this thread. Do this before + // creating RecordReader, because RecordReader's constructor might read some bytes + val bytesReadCallback = inputMetrics.bytesReadCallback.orElse { + split.serializableHadoopSplit.value match { + case _: FileSplit | _: CombineFileSplit => + SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() + case _ => None + } + } + inputMetrics.setBytesReadCallback(bytesReadCallback) + + val attemptId = newTaskAttemptID(jobTrackerId, id, isMap = true, split.index, 0) + val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) + val format = inputFormatClass.newInstance + format match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => + } + val reader = format.createRecordReader( + split.serializableHadoopSplit.value, hadoopAttemptContext) + reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) + + // Register an on-task-completion callback to close the input stream. + context.addTaskCompletionListener(context => close()) + var havePair = false + var finished = false + var recordsSinceMetricsUpdate = 0 + + override def hasNext: Boolean = { + if (!finished && !havePair) { + finished = !reader.nextKeyValue + havePair = !finished + } + !finished + } + + override def next(): (K, V) = { + if (!hasNext) { + throw new java.util.NoSuchElementException("End of stream") + } + havePair = false + if (!finished) { + inputMetrics.incRecordsRead(1) + } + (reader.getCurrentKey, reader.getCurrentValue) + } + + private def close() { + try { + reader.close() + if (bytesReadCallback.isDefined) { + inputMetrics.updateBytesRead() + } else if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit] || + split.serializableHadoopSplit.value.isInstanceOf[CombineFileSplit]) { + // If we can't get the bytes read from the FS stats, fall back to the split size, + // which may be inaccurate. + try { + inputMetrics.incBytesRead(split.serializableHadoopSplit.value.getLength) + } catch { + case e: java.io.IOException => + logWarning("Unable to get input size to set InputMetrics for task", e) + } + } + } catch { + case e: Exception => { + if (!Utils.inShutdown()) { + logWarning("Exception in RecordReader.close()", e) + } + } + } + } + } + new InterruptibleIterator(context, iter) + } + + /** Maps over a partition, providing the InputSplit that was used as the base of the partition. */ + @DeveloperApi + def mapPartitionsWithInputSplit[U: ClassTag]( + f: (InputSplit, Iterator[(K, V)]) => Iterator[U], + preservesPartitioning: Boolean = false): RDD[U] = { + new NewHadoopMapPartitionsWithSplitRDD(this, f, preservesPartitioning) + } + + override def getPreferredLocations(hsplit: SparkPartition): Seq[String] = { + val split = hsplit.asInstanceOf[SqlNewHadoopPartition].serializableHadoopSplit.value + val locs = HadoopRDD.SPLIT_INFO_REFLECTIONS match { + case Some(c) => + try { + val infos = c.newGetLocationInfo.invoke(split).asInstanceOf[Array[AnyRef]] + Some(HadoopRDD.convertSplitLocationInfo(infos)) + } catch { + case e : Exception => + logDebug("Failed to use InputSplit#getLocationInfo.", e) + None + } + case None => None + } + locs.getOrElse(split.getLocations.filter(_ != "localhost")) + } + + override def persist(storageLevel: StorageLevel): this.type = { + if (storageLevel.deserialized) { + logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" + + " behavior because Hadoop's RecordReader reuses the same Writable object for all records." + + " Use a map transformation to make copies of the records.") + } + super.persist(storageLevel) + } +} + +private[spark] object SqlNewHadoopRDD { + /** + * Analogous to [[org.apache.spark.rdd.MapPartitionsRDD]], but passes in an InputSplit to + * the given function rather than the index of the partition. + */ + private[spark] class NewHadoopMapPartitionsWithSplitRDD[U: ClassTag, T: ClassTag]( + prev: RDD[T], + f: (InputSplit, Iterator[T]) => Iterator[U], + preservesPartitioning: Boolean = false) + extends RDD[U](prev) { + + override val partitioner = if (preservesPartitioning) firstParent[T].partitioner else None + + override def getPartitions: Array[SparkPartition] = firstParent[T].partitions + + override def compute(split: SparkPartition, context: TaskContext): Iterator[U] = { + val partition = split.asInstanceOf[SqlNewHadoopPartition] + val inputSplit = partition.serializableHadoopSplit.value + f(inputSplit, firstParent[T].iterator(split, context)) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 9b52d1be3df2d..6a917bf38b139 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -25,7 +25,9 @@ import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD +import org.apache.spark.SerializableWritable import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection @@ -484,7 +486,8 @@ abstract class HadoopFsRelation private[sql](maybePartitionSpec: Option[Partitio private[sources] final def buildScan( requiredColumns: Array[String], filters: Array[Filter], - inputPaths: Array[String]): RDD[Row] = { + inputPaths: Array[String], + broadcastedConf: Broadcast[SerializableWritable[Configuration]]): RDD[Row] = { val inputStatuses = inputPaths.flatMap { input => val path = new Path(input) @@ -499,7 +502,7 @@ abstract class HadoopFsRelation private[sql](maybePartitionSpec: Option[Partitio } } - buildScan(requiredColumns, filters, inputStatuses) + buildScan(requiredColumns, filters, inputStatuses, broadcastedConf) } /** @@ -583,6 +586,34 @@ abstract class HadoopFsRelation private[sql](maybePartitionSpec: Option[Partitio buildScan(requiredColumns, inputFiles) } + /** + * For a non-partitioned relation, this method builds an `RDD[Row]` containing all rows within + * this relation. For partitioned relations, this method is called for each selected partition, + * and builds an `RDD[Row]` containing all rows within that single partition. + * + * Note: This interface is subject to change in future. + * + * @param requiredColumns Required columns. + * @param filters Candidate filters to be pushed down. The actual filter should be the conjunction + * of all `filters`. The pushed down filters are currently purely an optimization as they + * will all be evaluated again. This means it is safe to use them with methods that produce + * false positives such as filtering partitions based on a bloom filter. + * @param inputFiles For a non-partitioned relation, it contains paths of all data files in the + * relation. For a partitioned relation, it contains paths of all data files in a single + * selected partition. + * @param broadcastedConf A shared broadcast Hadoop Configuration, which can be used to reduce the + * overhead of broadcasting the Configuration for every Hadoop RDD. + * + * @since 1.4.0 + */ + private[sql] def buildScan( + requiredColumns: Array[String], + filters: Array[Filter], + inputFiles: Array[FileStatus], + broadcastedConf: Broadcast[SerializableWritable[Configuration]]): RDD[Row] = { + buildScan(requiredColumns, filters, inputFiles) + } + /** * Prepares a write job and returns an [[OutputWriterFactory]]. Client side job preparation can * be put here. For example, user defined output committer can be configured here From 2ad4837cfa66fcedc96b0819a8c2f4c3d70b0aaa Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 20 May 2015 12:50:06 -0700 Subject: [PATCH 07/17] [SPARK-7537] [MLLIB] spark.mllib API updates Minor updates to the spark.mllib APIs: 1. Add `DeveloperApi` to `PMMLExportable` and add `Experimental` to `toPMML` methods. 2. Mention `RankingMetrics.of` in the `RankingMetrics` constructor. Author: Xiangrui Meng Closes #6280 from mengxr/SPARK-7537 and squashes the following commits: 1bd2583 [Xiangrui Meng] organize imports 94afa7a [Xiangrui Meng] mark all toPMML methods experimental 4c40da1 [Xiangrui Meng] mention the factory method for RankingMetrics for Java users 88c62d0 [Xiangrui Meng] add DeveloperApi to PMMLExportable --- .../spark/mllib/evaluation/RankingMetrics.scala | 2 ++ .../org/apache/spark/mllib/pmml/PMMLExportable.scala | 11 +++++++++++ 2 files changed, 13 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala index b9b54b93c27fa..5b5a2a1450f7f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala @@ -31,6 +31,8 @@ import org.apache.spark.rdd.RDD * ::Experimental:: * Evaluator for ranking algorithms. * + * Java users should use [[RankingMetrics$.of]] to create a [[RankingMetrics]] instance. + * * @param predictionAndLabels an RDD of (predicted ranking, ground truth set) pairs. */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala index 354e90f3eeaa6..5e882d4ebb10b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala @@ -23,13 +23,16 @@ import javax.xml.transform.stream.StreamResult import org.jpmml.model.JAXBUtil import org.apache.spark.SparkContext +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.mllib.pmml.export.PMMLModelExportFactory /** + * :: DeveloperApi :: * Export model to the PMML format * Predictive Model Markup Language (PMML) is an XML-based file format * developed by the Data Mining Group (www.dmg.org). */ +@DeveloperApi trait PMMLExportable { /** @@ -41,30 +44,38 @@ trait PMMLExportable { } /** + * :: Experimental :: * Export the model to a local file in PMML format */ + @Experimental def toPMML(localPath: String): Unit = { toPMML(new StreamResult(new File(localPath))) } /** + * :: Experimental :: * Export the model to a directory on a distributed file system in PMML format */ + @Experimental def toPMML(sc: SparkContext, path: String): Unit = { val pmml = toPMML() sc.parallelize(Array(pmml), 1).saveAsTextFile(path) } /** + * :: Experimental :: * Export the model to the OutputStream in PMML format */ + @Experimental def toPMML(outputStream: OutputStream): Unit = { toPMML(new StreamResult(outputStream)) } /** + * :: Experimental :: * Export the model to a String in PMML format */ + @Experimental def toPMML(): String = { val writer = new StringWriter toPMML(new StreamResult(writer)) From 829f1d95bac9153e7b646fbc0d55566ecf896200 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Wed, 20 May 2015 13:10:30 -0700 Subject: [PATCH 08/17] [SPARK-7579] [ML] [DOC] User guide update for OneHotEncoder Author: Sandy Ryza Closes #6126 from sryza/sandy-spark-7579 and squashes the following commits: 5af803d [Sandy Ryza] SPARK-7579 [MLLIB] User guide update for OneHotEncoder --- docs/ml-features.md | 95 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 95 insertions(+) diff --git a/docs/ml-features.md b/docs/ml-features.md index 63ea3e5db7ac9..235029d71fadd 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -440,5 +440,100 @@ for expanded in polyDF.select("polyFeatures").take(3): +## OneHotEncoder + +[One-hot encoding](http://en.wikipedia.org/wiki/One-hot) maps a column of label indices to a column of binary vectors, with at most a single one-value. This encoding allows algorithms which expect continuous features, such as Logistic Regression, to use categorical features + +
+
+{% highlight scala %} +import org.apache.spark.ml.feature.{OneHotEncoder, StringIndexer} + +val df = sqlContext.createDataFrame(Seq( + (0, "a"), + (1, "b"), + (2, "c"), + (3, "a"), + (4, "a"), + (5, "c") +)).toDF("id", "category") + +val indexer = new StringIndexer() + .setInputCol("category") + .setOutputCol("categoryIndex") + .fit(df) +val indexed = indexer.transform(df) + +val encoder = new OneHotEncoder().setInputCol("categoryIndex"). + setOutputCol("categoryVec") +val encoded = encoder.transform(indexed) +encoded.select("id", "categoryVec").foreach(println) +{% endhighlight %} +
+ +
+{% highlight java %} +import com.google.common.collect.Lists; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.ml.feature.OneHotEncoder; +import org.apache.spark.ml.feature.StringIndexer; +import org.apache.spark.ml.feature.StringIndexerModel; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +JavaRDD jrdd = jsc.parallelize(Lists.newArrayList( + RowFactory.create(0, "a"), + RowFactory.create(1, "b"), + RowFactory.create(2, "c"), + RowFactory.create(3, "a"), + RowFactory.create(4, "a"), + RowFactory.create(5, "c") +)); +StructType schema = new StructType(new StructField[]{ + new StructField("id", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("category", DataTypes.StringType, false, Metadata.empty()) +}); +DataFrame df = sqlContext.createDataFrame(jrdd, schema); +StringIndexerModel indexer = new StringIndexer() + .setInputCol("category") + .setOutputCol("categoryIndex") + .fit(df); +DataFrame indexed = indexer.transform(df); + +OneHotEncoder encoder = new OneHotEncoder() + .setInputCol("categoryIndex") + .setOutputCol("categoryVec"); +DataFrame encoded = encoder.transform(indexed); +{% endhighlight %} +
+ +
+{% highlight python %} +from pyspark.ml.feature import OneHotEncoder, StringIndexer + +df = sqlContext.createDataFrame([ + (0, "a"), + (1, "b"), + (2, "c"), + (3, "a"), + (4, "a"), + (5, "c") +], ["id", "category"]) + +stringIndexer = StringIndexer(inputCol="category", outputCol="categoryIndex") +model = stringIndexer.fit(df) +indexed = model.transform(df) +encoder = OneHotEncoder(includeFirst=False, inputCol="categoryIndex", outputCol="categoryVec") +encoded = encoder.transform(indexed) +{% endhighlight %} +
+
+ # Feature Selectors From 6338c40da61de045485c51aa11a5b1e425d22144 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 20 May 2015 13:39:04 -0700 Subject: [PATCH 09/17] Revert "[SPARK-7320] [SQL] Add Cube / Rollup for dataframe" This reverts commit 10698e1131f665addb454cd498669920699a91b2. --- .../org/apache/spark/sql/DataFrame.scala | 104 +----------------- .../org/apache/spark/sql/GroupedData.scala | 92 +++++----------- .../hive/HiveDataFrameAnalyticsSuite.scala | 62 ----------- 3 files changed, 28 insertions(+), 230 deletions(-) delete mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index d78b4c2f8909c..adad85806d1ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -685,53 +685,7 @@ class DataFrame private[sql]( * @since 1.3.0 */ @scala.annotation.varargs - def groupBy(cols: Column*): GroupedData = { - GroupedData(this, cols.map(_.expr), GroupedData.GroupByType) - } - - /** - * Create a multi-dimensional rollup for the current [[DataFrame]] using the specified columns, - * so we can run aggregation on them. - * See [[GroupedData]] for all the available aggregate functions. - * - * {{{ - * // Compute the average for all numeric columns rolluped by department and group. - * df.rollup($"department", $"group").avg() - * - * // Compute the max age and average salary, rolluped by department and gender. - * df.rollup($"department", $"gender").agg(Map( - * "salary" -> "avg", - * "age" -> "max" - * )) - * }}} - * @group dfops - * @since 1.4.0 - */ - @scala.annotation.varargs - def rollup(cols: Column*): GroupedData = { - GroupedData(this, cols.map(_.expr), GroupedData.RollupType) - } - - /** - * Create a multi-dimensional cube for the current [[DataFrame]] using the specified columns, - * so we can run aggregation on them. - * See [[GroupedData]] for all the available aggregate functions. - * - * {{{ - * // Compute the average for all numeric columns cubed by department and group. - * df.cube($"department", $"group").avg() - * - * // Compute the max age and average salary, cubed by department and gender. - * df.cube($"department", $"gender").agg(Map( - * "salary" -> "avg", - * "age" -> "max" - * )) - * }}} - * @group dfops - * @since 1.4.0 - */ - @scala.annotation.varargs - def cube(cols: Column*): GroupedData = GroupedData(this, cols.map(_.expr), GroupedData.CubeType) + def groupBy(cols: Column*): GroupedData = new GroupedData(this, cols.map(_.expr)) /** * Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them. @@ -756,61 +710,7 @@ class DataFrame private[sql]( @scala.annotation.varargs def groupBy(col1: String, cols: String*): GroupedData = { val colNames: Seq[String] = col1 +: cols - GroupedData(this, colNames.map(colName => resolve(colName)), GroupedData.GroupByType) - } - - /** - * Create a multi-dimensional rollup for the current [[DataFrame]] using the specified columns, - * so we can run aggregation on them. - * See [[GroupedData]] for all the available aggregate functions. - * - * This is a variant of rollup that can only group by existing columns using column names - * (i.e. cannot construct expressions). - * - * {{{ - * // Compute the average for all numeric columns rolluped by department and group. - * df.rollup("department", "group").avg() - * - * // Compute the max age and average salary, rolluped by department and gender. - * df.rollup($"department", $"gender").agg(Map( - * "salary" -> "avg", - * "age" -> "max" - * )) - * }}} - * @group dfops - * @since 1.4.0 - */ - @scala.annotation.varargs - def rollup(col1: String, cols: String*): GroupedData = { - val colNames: Seq[String] = col1 +: cols - GroupedData(this, colNames.map(colName => resolve(colName)), GroupedData.RollupType) - } - - /** - * Create a multi-dimensional cube for the current [[DataFrame]] using the specified columns, - * so we can run aggregation on them. - * See [[GroupedData]] for all the available aggregate functions. - * - * This is a variant of cube that can only group by existing columns using column names - * (i.e. cannot construct expressions). - * - * {{{ - * // Compute the average for all numeric columns cubed by department and group. - * df.cube("department", "group").avg() - * - * // Compute the max age and average salary, cubed by department and gender. - * df.cube($"department", $"gender").agg(Map( - * "salary" -> "avg", - * "age" -> "max" - * )) - * }}} - * @group dfops - * @since 1.4.0 - */ - @scala.annotation.varargs - def cube(col1: String, cols: String*): GroupedData = { - val colNames: Seq[String] = col1 +: cols - GroupedData(this, colNames.map(colName => resolve(colName)), GroupedData.CubeType) + new GroupedData(this, colNames.map(colName => resolve(colName))) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala index f730e4ae00e2b..1381b9f1a6080 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala @@ -23,40 +23,9 @@ import scala.language.implicitConversions import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.analysis.Star import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{Rollup, Cube, Aggregate} +import org.apache.spark.sql.catalyst.plans.logical.Aggregate import org.apache.spark.sql.types.NumericType -/** - * Companion object for GroupedData - */ -private[sql] object GroupedData { - def apply( - df: DataFrame, - groupingExprs: Seq[Expression], - groupType: GroupType): GroupedData = { - new GroupedData(df, groupingExprs, groupType: GroupType) - } - - /** - * The Grouping Type - */ - trait GroupType - - /** - * To indicate it's the GroupBy - */ - object GroupByType extends GroupType - - /** - * To indicate it's the CUBE - */ - object CubeType extends GroupType - - /** - * To indicate it's the ROLLUP - */ - object RollupType extends GroupType -} /** * :: Experimental :: @@ -65,37 +34,19 @@ private[sql] object GroupedData { * @since 1.3.0 */ @Experimental -class GroupedData protected[sql]( - df: DataFrame, - groupingExprs: Seq[Expression], - private val groupType: GroupedData.GroupType) { +class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) { - private[this] def toDF(aggExprs: Seq[NamedExpression]): DataFrame = { - val aggregates = if (df.sqlContext.conf.dataFrameRetainGroupColumns) { - val retainedExprs = groupingExprs.map { - case expr: NamedExpression => expr - case expr: Expression => Alias(expr, expr.prettyString)() - } - retainedExprs ++ aggExprs - } else { - aggExprs - } - - groupType match { - case GroupedData.GroupByType => - DataFrame( - df.sqlContext, Aggregate(groupingExprs, aggregates, df.logicalPlan)) - case GroupedData.RollupType => - DataFrame( - df.sqlContext, Rollup(groupingExprs, df.logicalPlan, aggregates)) - case GroupedData.CubeType => - DataFrame( - df.sqlContext, Cube(groupingExprs, df.logicalPlan, aggregates)) + private[sql] implicit def toDF(aggExprs: Seq[NamedExpression]): DataFrame = { + val namedGroupingExprs = groupingExprs.map { + case expr: NamedExpression => expr + case expr: Expression => Alias(expr, expr.prettyString)() } + DataFrame( + df.sqlContext, Aggregate(groupingExprs, namedGroupingExprs ++ aggExprs, df.logicalPlan)) } private[this] def aggregateNumericColumns(colNames: String*)(f: Expression => Expression) - : DataFrame = { + : Seq[NamedExpression] = { val columnExprs = if (colNames.isEmpty) { // No columns specified. Use all numeric columns. @@ -112,10 +63,10 @@ class GroupedData protected[sql]( namedExpr } } - toDF(columnExprs.map { c => + columnExprs.map { c => val a = f(c) Alias(a, a.prettyString)() - }) + } } private[this] def strToExpr(expr: String): (Expression => Expression) = { @@ -168,10 +119,10 @@ class GroupedData protected[sql]( * @since 1.3.0 */ def agg(exprs: Map[String, String]): DataFrame = { - toDF(exprs.map { case (colName, expr) => + exprs.map { case (colName, expr) => val a = strToExpr(expr)(df(colName).expr) Alias(a, a.prettyString)() - }.toSeq) + }.toSeq } /** @@ -224,10 +175,19 @@ class GroupedData protected[sql]( */ @scala.annotation.varargs def agg(expr: Column, exprs: Column*): DataFrame = { - toDF((expr +: exprs).map(_.expr).map { + val aggExprs = (expr +: exprs).map(_.expr).map { case expr: NamedExpression => expr case expr: Expression => Alias(expr, expr.prettyString)() - }) + } + if (df.sqlContext.conf.dataFrameRetainGroupColumns) { + val retainedExprs = groupingExprs.map { + case expr: NamedExpression => expr + case expr: Expression => Alias(expr, expr.prettyString)() + } + DataFrame(df.sqlContext, Aggregate(groupingExprs, retainedExprs ++ aggExprs, df.logicalPlan)) + } else { + DataFrame(df.sqlContext, Aggregate(groupingExprs, aggExprs, df.logicalPlan)) + } } /** @@ -236,7 +196,7 @@ class GroupedData protected[sql]( * * @since 1.3.0 */ - def count(): DataFrame = toDF(Seq(Alias(Count(Literal(1)), "count")())) + def count(): DataFrame = Seq(Alias(Count(Literal(1)), "count")()) /** * Compute the average value for each numeric columns for each group. This is an alias for `avg`. @@ -296,5 +256,5 @@ class GroupedData protected[sql]( @scala.annotation.varargs def sum(colNames: String*): DataFrame = { aggregateNumericColumns(colNames:_*)(Sum) - } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala deleted file mode 100644 index 3ad05f482504c..0000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala +++ /dev/null @@ -1,62 +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.sql.hive - -import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.hive.test.TestHive -import org.apache.spark.sql.hive.test.TestHive._ -import org.apache.spark.sql.hive.test.TestHive.implicits._ - -case class TestData2Int(a: Int, b: Int) - -// TODO ideally we should put the test suite into the package `sql`, as -// `hive` package is optional in compiling, however, `SQLContext.sql` doesn't -// support the `cube` or `rollup` yet. -class HiveDataFrameAnalyticsSuite extends QueryTest { - val testData = - TestHive.sparkContext.parallelize( - TestData2Int(1, 2) :: - TestData2Int(2, 4) :: Nil).toDF() - - testData.registerTempTable("mytable") - - test("rollup") { - checkAnswer( - testData.rollup($"a" + $"b", $"b").agg(sum($"a" - $"b")), - sql("select a + b, b, sum(a - b) from mytable group by a + b, b with rollup").collect() - ) - - checkAnswer( - testData.rollup("a", "b").agg(sum("b")), - sql("select a, b, sum(b) from mytable group by a, b with rollup").collect() - ) - } - - test("cube") { - checkAnswer( - testData.cube($"a" + $"b", $"b").agg(sum($"a" - $"b")), - sql("select a + b, b, sum(a - b) from mytable group by a + b, b with cube").collect() - ) - - checkAnswer( - testData.cube("a", "b").agg(sum("b")), - sql("select a, b, sum(b) from mytable group by a, b with cube").collect() - ) - } -} From 191ee474527530246ac3164ae9631e01bdd1e647 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 20 May 2015 15:16:12 -0700 Subject: [PATCH 10/17] [SPARK-7511] [MLLIB] pyspark ml seed param should be random by default or 42 is quite funny but not very random Author: Holden Karau Closes #6139 from holdenk/SPARK-7511-pyspark-ml-seed-param-should-be-random-by-default-or-42-is-quite-funny-but-not-very-random and squashes the following commits: 591f8e5 [Holden Karau] specify old seed for doc tests 2470004 [Holden Karau] Fix a bunch of seeds with default values to have None as the default which will then result in using the hash of the class name cbad96d [Holden Karau] Add the setParams function that is used in the real code 423b8d7 [Holden Karau] Switch the test code to behave slightly more like production code. also don't check the param map value only check for key existence 140d25d [Holden Karau] remove extra space 926165a [Holden Karau] Add some missing newlines for pep8 style 8616751 [Holden Karau] merge in master 58532e6 [Holden Karau] its the __name__ method, also treat None values as not set 56ef24a [Holden Karau] fix test and regenerate base afdaa5c [Holden Karau] make sure different classes have different results 68eb528 [Holden Karau] switch default seed to hash of type of self 89c4611 [Holden Karau] Merge branch 'master' into SPARK-7511-pyspark-ml-seed-param-should-be-random-by-default-or-42-is-quite-funny-but-not-very-random 31cd96f [Holden Karau] specify the seed to randomforestregressor test e1b947f [Holden Karau] Style fixes ce90ec8 [Holden Karau] merge in master bcdf3c9 [Holden Karau] update docstring seeds to none and some other default seeds from 42 65eba21 [Holden Karau] pep8 fixes 0e3797e [Holden Karau] Make seed default to random in more places 213a543 [Holden Karau] Simplify the generated code to only include set default if there is a default rather than having None is note None in the generated code 1ff17c2 [Holden Karau] Make the seed random for HasSeed in python --- python/pyspark/ml/classification.py | 12 ++-- python/pyspark/ml/feature.py | 10 +-- python/pyspark/ml/param/__init__.py | 2 +- .../ml/param/_shared_params_code_gen.py | 9 +-- python/pyspark/ml/param/shared.py | 37 ++-------- python/pyspark/ml/recommendation.py | 10 +-- python/pyspark/ml/regression.py | 13 ++-- python/pyspark/ml/tests.py | 67 +++++++++++++++++-- 8 files changed, 96 insertions(+), 64 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 4e645519c47c7..7abbde8b260eb 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -292,7 +292,7 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed") >>> si_model = stringIndexer.fit(df) >>> td = si_model.transform(df) - >>> rf = RandomForestClassifier(numTrees=2, maxDepth=2, labelCol="indexed") + >>> rf = RandomForestClassifier(numTrees=2, maxDepth=2, labelCol="indexed", seed=42) >>> model = rf.fit(td) >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction @@ -319,12 +319,12 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", - numTrees=20, featureSubsetStrategy="auto", seed=42): + numTrees=20, featureSubsetStrategy="auto", seed=None): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", \ - numTrees=20, featureSubsetStrategy="auto", seed=42) + numTrees=20, featureSubsetStrategy="auto", seed=None) """ super(RandomForestClassifier, self).__init__() self._java_obj = self._new_java_obj( @@ -347,7 +347,7 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred "The number of features to consider for splits at each tree node. Supported " + "options: " + ", ".join(RandomForestParams.supportedFeatureSubsetStrategies)) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=42, + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=None, impurity="gini", numTrees=20, featureSubsetStrategy="auto") kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @@ -355,12 +355,12 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred @keyword_only def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=42, + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=None, impurity="gini", numTrees=20, featureSubsetStrategy="auto"): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=42, \ + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=None, \ impurity="gini", numTrees=20, featureSubsetStrategy="auto") Sets params for linear classification. """ diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index c8115cb5bcf63..5511dceb70419 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -876,10 +876,10 @@ class Word2Vec(JavaEstimator, HasStepSize, HasMaxIter, HasSeed, HasInputCol, Has @keyword_only def __init__(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, - seed=42, inputCol=None, outputCol=None): + seed=None, inputCol=None, outputCol=None): """ __init__(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, \ - seed=42, inputCol=None, outputCol=None) + seed=None, inputCol=None, outputCol=None) """ super(Word2Vec, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Word2Vec", self.uid) @@ -891,15 +891,15 @@ def __init__(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, "the minimum number of times a token must appear to be included " + "in the word2vec model's vocabulary") self._setDefault(vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, - seed=42) + seed=None) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only def setParams(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, - seed=42, inputCol=None, outputCol=None): + seed=None, inputCol=None, outputCol=None): """ - setParams(self, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, seed=42, \ + setParams(self, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, seed=None, \ inputCol=None, outputCol=None) Sets params for this Word2Vec. """ diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 67fb6e3dc74fb..7845536161e07 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -147,7 +147,7 @@ def hasParam(self, paramName): def getOrDefault(self, param): """ Gets the value of a param in the user-supplied param map or its - default value. Raises an error if either is set. + default value. Raises an error if neither is set. """ param = self._resolveParam(param) if param in self._paramMap: diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index 91e45ec373518..ccb929af184b8 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -56,9 +56,10 @@ def _gen_param_header(name, doc, defaultValueStr): def __init__(self): super(Has$Name, self).__init__() #: param for $doc - self.$name = Param(self, "$name", "$doc") - if $defaultValueStr is not None: - self._setDefault($name=$defaultValueStr)''' + self.$name = Param(self, "$name", "$doc")''' + if defaultValueStr is not None: + template += ''' + self._setDefault($name=$defaultValueStr)''' Name = name[0].upper() + name[1:] return template \ @@ -118,7 +119,7 @@ def get$Name(self): ("outputCol", "output column name", None), ("numFeatures", "number of features", None), ("checkpointInterval", "checkpoint interval (>= 1)", None), - ("seed", "random seed", None), + ("seed", "random seed", "hash(type(self).__name__)"), ("tol", "the convergence tolerance for iterative algorithms", None), ("stepSize", "Step size to be used for each iteration of optimization.", None)] code = [] diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index a5dc9b7ef29ed..0b93788899124 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -32,8 +32,6 @@ def __init__(self): super(HasMaxIter, self).__init__() #: param for max number of iterations (>= 0) self.maxIter = Param(self, "maxIter", "max number of iterations (>= 0)") - if None is not None: - self._setDefault(maxIter=None) def setMaxIter(self, value): """ @@ -61,8 +59,6 @@ def __init__(self): super(HasRegParam, self).__init__() #: param for regularization parameter (>= 0) self.regParam = Param(self, "regParam", "regularization parameter (>= 0)") - if None is not None: - self._setDefault(regParam=None) def setRegParam(self, value): """ @@ -90,8 +86,7 @@ def __init__(self): super(HasFeaturesCol, self).__init__() #: param for features column name self.featuresCol = Param(self, "featuresCol", "features column name") - if 'features' is not None: - self._setDefault(featuresCol='features') + self._setDefault(featuresCol='features') def setFeaturesCol(self, value): """ @@ -119,8 +114,7 @@ def __init__(self): super(HasLabelCol, self).__init__() #: param for label column name self.labelCol = Param(self, "labelCol", "label column name") - if 'label' is not None: - self._setDefault(labelCol='label') + self._setDefault(labelCol='label') def setLabelCol(self, value): """ @@ -148,8 +142,7 @@ def __init__(self): super(HasPredictionCol, self).__init__() #: param for prediction column name self.predictionCol = Param(self, "predictionCol", "prediction column name") - if 'prediction' is not None: - self._setDefault(predictionCol='prediction') + self._setDefault(predictionCol='prediction') def setPredictionCol(self, value): """ @@ -177,8 +170,7 @@ def __init__(self): super(HasProbabilityCol, self).__init__() #: param for Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities. self.probabilityCol = Param(self, "probabilityCol", "Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities.") - if 'probability' is not None: - self._setDefault(probabilityCol='probability') + self._setDefault(probabilityCol='probability') def setProbabilityCol(self, value): """ @@ -206,8 +198,7 @@ def __init__(self): super(HasRawPredictionCol, self).__init__() #: param for raw prediction (a.k.a. confidence) column name self.rawPredictionCol = Param(self, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name") - if 'rawPrediction' is not None: - self._setDefault(rawPredictionCol='rawPrediction') + self._setDefault(rawPredictionCol='rawPrediction') def setRawPredictionCol(self, value): """ @@ -235,8 +226,6 @@ def __init__(self): super(HasInputCol, self).__init__() #: param for input column name self.inputCol = Param(self, "inputCol", "input column name") - if None is not None: - self._setDefault(inputCol=None) def setInputCol(self, value): """ @@ -264,8 +253,6 @@ def __init__(self): super(HasInputCols, self).__init__() #: param for input column names self.inputCols = Param(self, "inputCols", "input column names") - if None is not None: - self._setDefault(inputCols=None) def setInputCols(self, value): """ @@ -293,8 +280,6 @@ def __init__(self): super(HasOutputCol, self).__init__() #: param for output column name self.outputCol = Param(self, "outputCol", "output column name") - if None is not None: - self._setDefault(outputCol=None) def setOutputCol(self, value): """ @@ -322,8 +307,6 @@ def __init__(self): super(HasNumFeatures, self).__init__() #: param for number of features self.numFeatures = Param(self, "numFeatures", "number of features") - if None is not None: - self._setDefault(numFeatures=None) def setNumFeatures(self, value): """ @@ -351,8 +334,6 @@ def __init__(self): super(HasCheckpointInterval, self).__init__() #: param for checkpoint interval (>= 1) self.checkpointInterval = Param(self, "checkpointInterval", "checkpoint interval (>= 1)") - if None is not None: - self._setDefault(checkpointInterval=None) def setCheckpointInterval(self, value): """ @@ -380,8 +361,7 @@ def __init__(self): super(HasSeed, self).__init__() #: param for random seed self.seed = Param(self, "seed", "random seed") - if None is not None: - self._setDefault(seed=None) + self._setDefault(seed=hash(type(self).__name__)) def setSeed(self, value): """ @@ -409,8 +389,6 @@ def __init__(self): super(HasTol, self).__init__() #: param for the convergence tolerance for iterative algorithms self.tol = Param(self, "tol", "the convergence tolerance for iterative algorithms") - if None is not None: - self._setDefault(tol=None) def setTol(self, value): """ @@ -438,8 +416,6 @@ def __init__(self): super(HasStepSize, self).__init__() #: param for Step size to be used for each iteration of optimization. self.stepSize = Param(self, "stepSize", "Step size to be used for each iteration of optimization.") - if None is not None: - self._setDefault(stepSize=None) def setStepSize(self, value): """ @@ -467,6 +443,7 @@ class DecisionTreeParams(Params): minInfoGain = Param(Params._dummy(), "minInfoGain", "Minimum information gain for a split to be considered at a tree node.") maxMemoryInMB = Param(Params._dummy(), "maxMemoryInMB", "Maximum memory in MB allocated to histogram aggregation.") cacheNodeIds = Param(Params._dummy(), "cacheNodeIds", "If false, the algorithm will pass trees to executors to match instances with nodes. If true, the algorithm will cache node IDs for each instance. Caching can speed up training of deeper trees.") + def __init__(self): super(DecisionTreeParams, self).__init__() diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index 39c2527543774..b3e0dd7abf681 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -89,11 +89,11 @@ class ALS(JavaEstimator, HasCheckpointInterval, HasMaxIter, HasPredictionCol, Ha @keyword_only def __init__(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, - implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=0, + implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=None, ratingCol="rating", nonnegative=False, checkpointInterval=10): """ __init__(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, \ - implicitPrefs=false, alpha=1.0, userCol="user", itemCol="item", seed=0, \ + implicitPrefs=false, alpha=1.0, userCol="user", itemCol="item", seed=None, \ ratingCol="rating", nonnegative=false, checkpointInterval=10) """ super(ALS, self).__init__() @@ -109,18 +109,18 @@ def __init__(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemB self.nonnegative = Param(self, "nonnegative", "whether to use nonnegative constraint for least squares") self._setDefault(rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, - implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=0, + implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=None, ratingCol="rating", nonnegative=False, checkpointInterval=10) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only def setParams(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, - implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=0, + implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=None, ratingCol="rating", nonnegative=False, checkpointInterval=10): """ setParams(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, \ - implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=0, \ + implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=None, \ ratingCol="rating", nonnegative=False, checkpointInterval=10) Sets params for ALS. """ diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index ff809cdafdf51..b139e27372d80 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -257,7 +257,7 @@ class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) - >>> rf = RandomForestRegressor(numTrees=2, maxDepth=2) + >>> rf = RandomForestRegressor(numTrees=2, maxDepth=2, seed=42) >>> model = rf.fit(df) >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction @@ -284,12 +284,13 @@ class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="variance", - numTrees=20, featureSubsetStrategy="auto", seed=42): + numTrees=20, featureSubsetStrategy="auto", seed=None): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, \ - impurity="variance", numTrees=20, featureSubsetStrategy="auto", seed=42) + impurity="variance", numTrees=20, \ + featureSubsetStrategy="auto", seed=None) """ super(RandomForestRegressor, self).__init__() self._java_obj = self._new_java_obj( @@ -312,7 +313,7 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred "The number of features to consider for splits at each tree node. Supported " + "options: " + ", ".join(RandomForestParams.supportedFeatureSubsetStrategies)) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=42, + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=None, impurity="variance", numTrees=20, featureSubsetStrategy="auto") kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @@ -320,12 +321,12 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred @keyword_only def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=42, + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=None, impurity="variance", numTrees=20, featureSubsetStrategy="auto"): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=42, \ + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=None, \ impurity="variance", numTrees=20, featureSubsetStrategy="auto") Sets params for linear regression. """ diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 10fe0ef8db38f..6adbf166f34a8 100644 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -33,7 +33,8 @@ from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase from pyspark.sql import DataFrame, SQLContext from pyspark.ml.param import Param, Params -from pyspark.ml.param.shared import HasMaxIter, HasInputCol +from pyspark.ml.param.shared import HasMaxIter, HasInputCol, HasSeed +from pyspark.ml.util import keyword_only from pyspark.ml import Estimator, Model, Pipeline, Transformer from pyspark.ml.feature import * from pyspark.mllib.linalg import DenseVector @@ -111,14 +112,46 @@ def test_pipeline(self): self.assertEqual(6, dataset.index) -class TestParams(HasMaxIter, HasInputCol): +class TestParams(HasMaxIter, HasInputCol, HasSeed): """ - A subclass of Params mixed with HasMaxIter and HasInputCol. + A subclass of Params mixed with HasMaxIter, HasInputCol and HasSeed. """ - - def __init__(self): + @keyword_only + def __init__(self, seed=None): super(TestParams, self).__init__() self._setDefault(maxIter=10) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, seed=None): + """ + setParams(self, seed=None) + Sets params for this test. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + +class OtherTestParams(HasMaxIter, HasInputCol, HasSeed): + """ + A subclass of Params mixed with HasMaxIter, HasInputCol and HasSeed. + """ + @keyword_only + def __init__(self, seed=None): + super(OtherTestParams, self).__init__() + self._setDefault(maxIter=10) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, seed=None): + """ + setParams(self, seed=None) + Sets params for this test. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) class ParamTests(PySparkTestCase): @@ -134,9 +167,10 @@ def test_params(self): testParams = TestParams() maxIter = testParams.maxIter inputCol = testParams.inputCol + seed = testParams.seed params = testParams.params - self.assertEqual(params, [inputCol, maxIter]) + self.assertEqual(params, [inputCol, maxIter, seed]) self.assertTrue(testParams.hasParam(maxIter)) self.assertTrue(testParams.hasDefault(maxIter)) @@ -154,10 +188,29 @@ def test_params(self): with self.assertRaises(KeyError): testParams.getInputCol() + # Since the default is normally random, set it to a known number for debug str + testParams._setDefault(seed=41) + testParams.setSeed(43) + self.assertEquals( testParams.explainParams(), "\n".join(["inputCol: input column name (undefined)", - "maxIter: max number of iterations (>= 0) (default: 10, current: 100)"])) + "maxIter: max number of iterations (>= 0) (default: 10, current: 100)", + "seed: random seed (default: 41, current: 43)"])) + + def test_hasseed(self): + noSeedSpecd = TestParams() + withSeedSpecd = TestParams(seed=42) + other = OtherTestParams() + # Check that we no longer use 42 as the magic number + self.assertNotEqual(noSeedSpecd.getSeed(), 42) + origSeed = noSeedSpecd.getSeed() + # Check that we only compute the seed once + self.assertEqual(noSeedSpecd.getSeed(), origSeed) + # Check that a specified seed is honored + self.assertEqual(withSeedSpecd.getSeed(), 42) + # Check that a different class has a different seed + self.assertNotEqual(other.getSeed(), noSeedSpecd.getSeed()) class FeatureTests(PySparkTestCase): From 9b84443dd43777e25b0b00468c61814fe6d26c23 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 20 May 2015 15:39:32 -0700 Subject: [PATCH 11/17] [SPARK-7237] [SPARK-7741] [CORE] [STREAMING] Clean more closures that need cleaning SPARK-7741 is the equivalent of SPARK-7237 in streaming. This is an alternative to #6268. Author: Andrew Or Closes #6269 from andrewor14/clean-moar and squashes the following commits: c51c9ab [Andrew Or] Add periods (trivial) 6c686ac [Andrew Or] Merge branch 'master' of github.com:apache/spark into clean-moar 79a435b [Andrew Or] Fix tests d18c9f9 [Andrew Or] Merge branch 'master' of github.com:apache/spark into clean-moar 65ef07b [Andrew Or] Fix tests? 4b487a3 [Andrew Or] Add tests for closures passed to DStream operations 328139b [Andrew Or] Do not forget foreachRDD 5431f61 [Andrew Or] Clean streaming closures 72b7b73 [Andrew Or] Clean core closures --- .../scala/org/apache/spark/SparkContext.scala | 4 +- .../apache/spark/rdd/PairRDDFunctions.scala | 5 +- .../spark/util/ClosureCleanerSuite.scala | 4 + .../spark/streaming/StreamingContext.scala | 2 +- .../spark/streaming/dstream/DStream.scala | 5 +- .../dstream/PairDStreamFunctions.scala | 30 ++- .../spark/streaming/DStreamClosureSuite.scala | 196 ++++++++++++++++++ .../spark/streaming/DStreamScopeSuite.scala | 22 +- .../spark/streaming/TestSuiteBase.scala | 18 ++ 9 files changed, 249 insertions(+), 37 deletions(-) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/DStreamClosureSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3fe3dc5e300e8..cf3820fcb6a35 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1159,8 +1159,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli kcf: () => WritableConverter[K], vcf: () => WritableConverter[V]): RDD[(K, V)] = { withScope { assertNotStopped() - val kc = kcf() - val vc = vcf() + val kc = clean(kcf)() + val vc = clean(vcf)() val format = classOf[SequenceFileInputFormat[Writable, Writable]] val writables = hadoopFile(path, format, kc.writableClass(km).asInstanceOf[Class[Writable]], 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 a6d5d2c94e17f..8653cdee1adee 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -296,6 +296,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * before sending results to a reducer, similarly to a "combiner" in MapReduce. */ def reduceByKeyLocally(func: (V, V) => V): Map[K, V] = self.withScope { + val cleanedF = self.sparkContext.clean(func) if (keyClass.isArray) { throw new SparkException("reduceByKeyLocally() does not support array keys") @@ -305,7 +306,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val map = new JHashMap[K, V] iter.foreach { pair => val old = map.get(pair._1) - map.put(pair._1, if (old == null) pair._2 else func(old, pair._2)) + map.put(pair._1, if (old == null) pair._2 else cleanedF(old, pair._2)) } Iterator(map) } : Iterator[JHashMap[K, V]] @@ -313,7 +314,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val mergeMaps = (m1: JHashMap[K, V], m2: JHashMap[K, V]) => { m2.foreach { pair => val old = m1.get(pair._1) - m1.put(pair._1, if (old == null) pair._2 else func(old, pair._2)) + m1.put(pair._1, if (old == null) pair._2 else cleanedF(old, pair._2)) } m1 } : JHashMap[K, V] diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index e41f6ee27764e..7b165fe28bdd3 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -112,6 +112,7 @@ class ClosureCleanerSuite extends FunSuite { expectCorrectException { TestUserClosuresActuallyCleaned.testAggregateByKey(pairRdd) } expectCorrectException { TestUserClosuresActuallyCleaned.testFoldByKey(pairRdd) } expectCorrectException { TestUserClosuresActuallyCleaned.testReduceByKey(pairRdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testReduceByKeyLocally(pairRdd) } expectCorrectException { TestUserClosuresActuallyCleaned.testMapValues(pairRdd) } expectCorrectException { TestUserClosuresActuallyCleaned.testFlatMapValues(pairRdd) } expectCorrectException { TestUserClosuresActuallyCleaned.testForeachAsync(rdd) } @@ -315,6 +316,9 @@ private object TestUserClosuresActuallyCleaned { } def testFoldByKey(rdd: RDD[(Int, Int)]): Unit = { rdd.foldByKey(0) { case (_, _) => return; 1 } } def testReduceByKey(rdd: RDD[(Int, Int)]): Unit = { rdd.reduceByKey { case (_, _) => return; 1 } } + def testReduceByKeyLocally(rdd: RDD[(Int, Int)]): Unit = { + rdd.reduceByKeyLocally { case (_, _) => return; 1 } + } def testMapValues(rdd: RDD[(Int, Int)]): Unit = { rdd.mapValues { _ => return; 1 } } def testFlatMapValues(rdd: RDD[(Int, Int)]): Unit = { rdd.flatMapValues { _ => return; Seq() } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 7f181bcecd4bf..fe614c4be590f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -255,7 +255,7 @@ class StreamingContext private[streaming] ( * * Note: Return statements are NOT allowed in the given body. */ - private[streaming] def withNamedScope[U](name: String)(body: => U): U = { + private def withNamedScope[U](name: String)(body: => U): U = { RDDOperationScope.withScope(sc, name, allowNesting = false, ignoreParent = false)(body) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 5977481e1f081..7c50a766a9bad 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -539,7 +539,7 @@ abstract class DStream[T: ClassTag] ( /** Return a new DStream containing only the elements that satisfy a predicate. */ def filter(filterFunc: T => Boolean): DStream[T] = ssc.withScope { - new FilteredDStream(this, filterFunc) + new FilteredDStream(this, context.sparkContext.clean(filterFunc)) } /** @@ -624,7 +624,8 @@ abstract class DStream[T: ClassTag] ( * 'this' DStream will be registered as an output stream and therefore materialized. */ def foreachRDD(foreachFunc: RDD[T] => Unit): Unit = ssc.withScope { - this.foreachRDD((r: RDD[T], t: Time) => foreachFunc(r)) + val cleanedF = context.sparkContext.clean(foreachFunc, false) + this.foreachRDD((r: RDD[T], t: Time) => cleanedF(r)) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index 884a8e8b52289..fda22eb6ec42e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -38,6 +38,8 @@ class PairDStreamFunctions[K, V](self: DStream[(K,V)]) { private[streaming] def ssc = self.ssc + private[streaming] def sparkContext = self.context.sparkContext + private[streaming] def defaultPartitioner(numPartitions: Int = self.ssc.sc.defaultParallelism) = { new HashPartitioner(numPartitions) } @@ -98,8 +100,7 @@ class PairDStreamFunctions[K, V](self: DStream[(K,V)]) def reduceByKey( reduceFunc: (V, V) => V, partitioner: Partitioner): DStream[(K, V)] = ssc.withScope { - val cleanedReduceFunc = ssc.sc.clean(reduceFunc) - combineByKey((v: V) => v, cleanedReduceFunc, cleanedReduceFunc, partitioner) + combineByKey((v: V) => v, reduceFunc, reduceFunc, partitioner) } /** @@ -113,7 +114,15 @@ class PairDStreamFunctions[K, V](self: DStream[(K,V)]) mergeCombiner: (C, C) => C, partitioner: Partitioner, mapSideCombine: Boolean = true): DStream[(K, C)] = ssc.withScope { - new ShuffledDStream[K, V, C](self, createCombiner, mergeValue, mergeCombiner, partitioner, + val cleanedCreateCombiner = sparkContext.clean(createCombiner) + val cleanedMergeValue = sparkContext.clean(mergeValue) + val cleanedMergeCombiner = sparkContext.clean(mergeCombiner) + new ShuffledDStream[K, V, C]( + self, + cleanedCreateCombiner, + cleanedMergeValue, + cleanedMergeCombiner, + partitioner, mapSideCombine) } @@ -264,10 +273,9 @@ class PairDStreamFunctions[K, V](self: DStream[(K,V)]) slideDuration: Duration, partitioner: Partitioner ): DStream[(K, V)] = ssc.withScope { - val cleanedReduceFunc = ssc.sc.clean(reduceFunc) - self.reduceByKey(cleanedReduceFunc, partitioner) + self.reduceByKey(reduceFunc, partitioner) .window(windowDuration, slideDuration) - .reduceByKey(cleanedReduceFunc, partitioner) + .reduceByKey(reduceFunc, partitioner) } /** @@ -385,8 +393,9 @@ class PairDStreamFunctions[K, V](self: DStream[(K,V)]) updateFunc: (Seq[V], Option[S]) => Option[S], partitioner: Partitioner ): DStream[(K, S)] = ssc.withScope { + val cleanedUpdateF = sparkContext.clean(updateFunc) val newUpdateFunc = (iterator: Iterator[(K, Seq[V], Option[S])]) => { - iterator.flatMap(t => updateFunc(t._2, t._3).map(s => (t._1, s))) + iterator.flatMap(t => cleanedUpdateF(t._2, t._3).map(s => (t._1, s))) } updateStateByKey(newUpdateFunc, partitioner, true) } @@ -428,8 +437,9 @@ class PairDStreamFunctions[K, V](self: DStream[(K,V)]) partitioner: Partitioner, initialRDD: RDD[(K, S)] ): DStream[(K, S)] = ssc.withScope { + val cleanedUpdateF = sparkContext.clean(updateFunc) val newUpdateFunc = (iterator: Iterator[(K, Seq[V], Option[S])]) => { - iterator.flatMap(t => updateFunc(t._2, t._3).map(s => (t._1, s))) + iterator.flatMap(t => cleanedUpdateF(t._2, t._3).map(s => (t._1, s))) } updateStateByKey(newUpdateFunc, partitioner, true, initialRDD) } @@ -463,7 +473,7 @@ class PairDStreamFunctions[K, V](self: DStream[(K,V)]) * 'this' DStream without changing the key. */ def mapValues[U: ClassTag](mapValuesFunc: V => U): DStream[(K, U)] = ssc.withScope { - new MapValuedDStream[K, V, U](self, mapValuesFunc) + new MapValuedDStream[K, V, U](self, sparkContext.clean(mapValuesFunc)) } /** @@ -473,7 +483,7 @@ class PairDStreamFunctions[K, V](self: DStream[(K,V)]) def flatMapValues[U: ClassTag]( flatMapValuesFunc: V => TraversableOnce[U] ): DStream[(K, U)] = ssc.withScope { - new FlatMapValuedDStream[K, V, U](self, flatMapValuesFunc) + new FlatMapValuedDStream[K, V, U](self, sparkContext.clean(flatMapValuesFunc)) } /** diff --git a/streaming/src/test/scala/org/apache/spark/streaming/DStreamClosureSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/DStreamClosureSuite.scala new file mode 100644 index 0000000000000..6a1dd6949b204 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/DStreamClosureSuite.scala @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +import java.io.NotSerializableException + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.{HashPartitioner, SparkContext, SparkException} +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.util.ReturnStatementInClosureException + +/** + * Test that closures passed to DStream operations are actually cleaned. + */ +class DStreamClosureSuite extends FunSuite with BeforeAndAfterAll { + private var ssc: StreamingContext = null + + override def beforeAll(): Unit = { + val sc = new SparkContext("local", "test") + ssc = new StreamingContext(sc, Seconds(1)) + } + + override def afterAll(): Unit = { + ssc.stop(stopSparkContext = true) + ssc = null + } + + test("user provided closures are actually cleaned") { + val dstream = new DummyInputDStream(ssc) + val pairDstream = dstream.map { i => (i, i) } + // DStream + testMap(dstream) + testFlatMap(dstream) + testFilter(dstream) + testMapPartitions(dstream) + testReduce(dstream) + testForeach(dstream) + testForeachRDD(dstream) + testTransform(dstream) + testTransformWith(dstream) + testReduceByWindow(dstream) + // PairDStreamFunctions + testReduceByKey(pairDstream) + testCombineByKey(pairDstream) + testReduceByKeyAndWindow(pairDstream) + testUpdateStateByKey(pairDstream) + testMapValues(pairDstream) + testFlatMapValues(pairDstream) + // StreamingContext + testTransform2(ssc, dstream) + } + + /** + * Verify that the expected exception is thrown. + * + * We use return statements as an indication that a closure is actually being cleaned. + * We expect closure cleaner to find the return statements in the user provided closures. + */ + private def expectCorrectException(body: => Unit): Unit = { + try { + body + } catch { + case rse: ReturnStatementInClosureException => // Success! + case e @ (_: NotSerializableException | _: SparkException) => + throw new TestException( + s"Expected ReturnStatementInClosureException, but got $e.\n" + + "This means the closure provided by user is not actually cleaned.") + } + } + + // DStream operations + private def testMap(ds: DStream[Int]): Unit = expectCorrectException { + ds.map { _ => return; 1 } + } + private def testFlatMap(ds: DStream[Int]): Unit = expectCorrectException { + ds.flatMap { _ => return; Seq.empty } + } + private def testFilter(ds: DStream[Int]): Unit = expectCorrectException { + ds.filter { _ => return; true } + } + private def testMapPartitions(ds: DStream[Int]): Unit = expectCorrectException { + ds.mapPartitions { _ => return; Seq.empty.toIterator } + } + private def testReduce(ds: DStream[Int]): Unit = expectCorrectException { + ds.reduce { case (_, _) => return; 1 } + } + private def testForeach(ds: DStream[Int]): Unit = { + val foreachF1 = (rdd: RDD[Int], t: Time) => return + val foreachF2 = (rdd: RDD[Int]) => return + expectCorrectException { ds.foreach(foreachF1) } + expectCorrectException { ds.foreach(foreachF2) } + } + private def testForeachRDD(ds: DStream[Int]): Unit = { + val foreachRDDF1 = (rdd: RDD[Int], t: Time) => return + val foreachRDDF2 = (rdd: RDD[Int]) => return + expectCorrectException { ds.foreachRDD(foreachRDDF1) } + expectCorrectException { ds.foreachRDD(foreachRDDF2) } + } + private def testTransform(ds: DStream[Int]): Unit = { + val transformF1 = (rdd: RDD[Int]) => { return; rdd } + val transformF2 = (rdd: RDD[Int], time: Time) => { return; rdd } + expectCorrectException { ds.transform(transformF1) } + expectCorrectException { ds.transform(transformF2) } + } + private def testTransformWith(ds: DStream[Int]): Unit = { + val transformF1 = (rdd1: RDD[Int], rdd2: RDD[Int]) => { return; rdd1 } + val transformF2 = (rdd1: RDD[Int], rdd2: RDD[Int], time: Time) => { return; rdd2 } + expectCorrectException { ds.transformWith(ds, transformF1) } + expectCorrectException { ds.transformWith(ds, transformF2) } + } + private def testReduceByWindow(ds: DStream[Int]): Unit = { + val reduceF = (_: Int, _: Int) => { return; 1 } + expectCorrectException { ds.reduceByWindow(reduceF, Seconds(1), Seconds(2)) } + expectCorrectException { ds.reduceByWindow(reduceF, reduceF, Seconds(1), Seconds(2)) } + } + + // PairDStreamFunctions operations + private def testReduceByKey(ds: DStream[(Int, Int)]): Unit = { + val reduceF = (_: Int, _: Int) => { return; 1 } + expectCorrectException { ds.reduceByKey(reduceF) } + expectCorrectException { ds.reduceByKey(reduceF, 5) } + expectCorrectException { ds.reduceByKey(reduceF, new HashPartitioner(5)) } + } + private def testCombineByKey(ds: DStream[(Int, Int)]): Unit = { + expectCorrectException { + ds.combineByKey[Int]( + { _: Int => return; 1 }, + { case (_: Int, _: Int) => return; 1 }, + { case (_: Int, _: Int) => return; 1 }, + new HashPartitioner(5) + ) + } + } + private def testReduceByKeyAndWindow(ds: DStream[(Int, Int)]): Unit = { + val reduceF = (_: Int, _: Int) => { return; 1 } + val filterF = (_: (Int, Int)) => { return; false } + expectCorrectException { ds.reduceByKeyAndWindow(reduceF, Seconds(1)) } + expectCorrectException { ds.reduceByKeyAndWindow(reduceF, Seconds(1), Seconds(2)) } + expectCorrectException { ds.reduceByKeyAndWindow(reduceF, Seconds(1), Seconds(2), 5) } + expectCorrectException { + ds.reduceByKeyAndWindow(reduceF, Seconds(1), Seconds(2), new HashPartitioner(5)) + } + expectCorrectException { ds.reduceByKeyAndWindow(reduceF, reduceF, Seconds(2)) } + expectCorrectException { + ds.reduceByKeyAndWindow( + reduceF, reduceF, Seconds(2), Seconds(3), new HashPartitioner(5), filterF) + } + } + private def testUpdateStateByKey(ds: DStream[(Int, Int)]): Unit = { + val updateF1 = (_: Seq[Int], _: Option[Int]) => { return; Some(1) } + val updateF2 = (_: Iterator[(Int, Seq[Int], Option[Int])]) => { return; Seq((1, 1)).toIterator } + val initialRDD = ds.ssc.sparkContext.emptyRDD[Int].map { i => (i, i) } + expectCorrectException { ds.updateStateByKey(updateF1) } + expectCorrectException { ds.updateStateByKey(updateF1, 5) } + expectCorrectException { ds.updateStateByKey(updateF1, new HashPartitioner(5)) } + expectCorrectException { + ds.updateStateByKey(updateF1, new HashPartitioner(5), initialRDD) + } + expectCorrectException { + ds.updateStateByKey(updateF2, new HashPartitioner(5), true) + } + expectCorrectException { + ds.updateStateByKey(updateF2, new HashPartitioner(5), true, initialRDD) + } + } + private def testMapValues(ds: DStream[(Int, Int)]): Unit = expectCorrectException { + ds.mapValues { _ => return; 1 } + } + private def testFlatMapValues(ds: DStream[(Int, Int)]): Unit = expectCorrectException { + ds.flatMapValues { _ => return; Seq.empty } + } + + // StreamingContext operations + private def testTransform2(ssc: StreamingContext, ds: DStream[Int]): Unit = { + val transformF = (rdds: Seq[RDD[_]], time: Time) => { return; ssc.sparkContext.emptyRDD[Int] } + expectCorrectException { ssc.transform(Seq(ds), transformF) } + } + +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala index 392933102097e..e3fb2ef130859 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala @@ -20,8 +20,8 @@ package org.apache.spark.streaming import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.apache.spark.SparkContext -import org.apache.spark.rdd.{RDD, RDDOperationScope} -import org.apache.spark.streaming.dstream.{DStream, InputDStream} +import org.apache.spark.rdd.RDDOperationScope +import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.ui.UIUtils /** @@ -170,21 +170,3 @@ class DStreamScopeSuite extends FunSuite with BeforeAndAfter with BeforeAndAfter } } - -/** - * A dummy stream that does absolutely nothing. - */ -private class DummyDStream(ssc: StreamingContext) extends DStream[Int](ssc) { - override def dependencies: List[DStream[Int]] = List.empty - override def slideDuration: Duration = Seconds(1) - override def compute(time: Time): Option[RDD[Int]] = Some(ssc.sc.emptyRDD[Int]) -} - -/** - * A dummy input stream that does absolutely nothing. - */ -private class DummyInputDStream(ssc: StreamingContext) extends InputDStream[Int](ssc) { - override def start(): Unit = { } - override def stop(): Unit = { } - override def compute(time: Time): Option[RDD[Int]] = Some(ssc.sc.emptyRDD[Int]) -} 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 4f70ae7f1f187..554cd30223f44 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -35,6 +35,24 @@ import org.apache.spark.streaming.dstream.{DStream, InputDStream, ForEachDStream import org.apache.spark.streaming.scheduler._ import org.apache.spark.util.{ManualClock, Utils} +/** + * A dummy stream that does absolutely nothing. + */ +private[streaming] class DummyDStream(ssc: StreamingContext) extends DStream[Int](ssc) { + override def dependencies: List[DStream[Int]] = List.empty + override def slideDuration: Duration = Seconds(1) + override def compute(time: Time): Option[RDD[Int]] = Some(ssc.sc.emptyRDD[Int]) +} + +/** + * A dummy input stream that does absolutely nothing. + */ +private[streaming] class DummyInputDStream(ssc: StreamingContext) extends InputDStream[Int](ssc) { + override def start(): Unit = { } + override def stop(): Unit = { } + override def compute(time: Time): Option[RDD[Int]] = Some(ssc.sc.emptyRDD[Int]) +} + /** * This is a input stream just for the testsuites. This is equivalent to a checkpointable, * replayable, reliable message queue like Kafka. It requires a sequence as input, and From 3c434cbfd0d6821e5bcf572be792b787a514018b Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 20 May 2015 16:21:23 -0700 Subject: [PATCH 12/17] [SPARK-7767] [STREAMING] Added test for checkpoint serialization in StreamingContext.start() Currently, the background checkpointing thread fails silently if the checkpoint is not serializable. It is hard to debug and therefore its best to fail fast at `start()` when checkpointing is enabled and the checkpoint is not serializable. Author: Tathagata Das Closes #6292 from tdas/SPARK-7767 and squashes the following commits: 51304e6 [Tathagata Das] Addressed comments. c35237b [Tathagata Das] Added test for checkpoint serialization in StreamingContext.start() --- .../serializer/SerializationDebugger.scala | 2 +- .../apache/spark/streaming/Checkpoint.scala | 70 +++++++++++-------- .../spark/streaming/StreamingContext.scala | 26 ++++++- .../streaming/StreamingContextSuite.scala | 27 +++++-- 4 files changed, 89 insertions(+), 36 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala b/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala index 5abfa467c0ec8..bb5db545531d2 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala @@ -27,7 +27,7 @@ import scala.util.control.NonFatal import org.apache.spark.Logging -private[serializer] object SerializationDebugger extends Logging { +private[spark] object SerializationDebugger extends Logging { /** * Improve the given NotSerializableException with the serialization path leading from the given diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 7bfae253c3a0c..d8dc4e4101664 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -102,6 +102,44 @@ object Checkpoint extends Logging { Seq.empty } } + + /** Serialize the checkpoint, or throw any exception that occurs */ + def serialize(checkpoint: Checkpoint, conf: SparkConf): Array[Byte] = { + val compressionCodec = CompressionCodec.createCodec(conf) + val bos = new ByteArrayOutputStream() + val zos = compressionCodec.compressedOutputStream(bos) + val oos = new ObjectOutputStream(zos) + Utils.tryWithSafeFinally { + oos.writeObject(checkpoint) + } { + oos.close() + } + bos.toByteArray + } + + /** Deserialize a checkpoint from the input stream, or throw any exception that occurs */ + def deserialize(inputStream: InputStream, conf: SparkConf): Checkpoint = { + val compressionCodec = CompressionCodec.createCodec(conf) + var ois: ObjectInputStreamWithLoader = null + Utils.tryWithSafeFinally { + + // ObjectInputStream uses the last defined user-defined class loader in the stack + // to find classes, which maybe the wrong class loader. Hence, a inherited version + // of ObjectInputStream is used to explicitly use the current thread's default class + // loader to find and load classes. This is a well know Java issue and has popped up + // in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627) + val zis = compressionCodec.compressedInputStream(inputStream) + ois = new ObjectInputStreamWithLoader(zis, + Thread.currentThread().getContextClassLoader) + val cp = ois.readObject.asInstanceOf[Checkpoint] + cp.validate() + cp + } { + if (ois != null) { + ois.close() + } + } + } } @@ -189,17 +227,10 @@ class CheckpointWriter( } def write(checkpoint: Checkpoint, clearCheckpointDataLater: Boolean) { - val bos = new ByteArrayOutputStream() - val zos = compressionCodec.compressedOutputStream(bos) - val oos = new ObjectOutputStream(zos) - Utils.tryWithSafeFinally { - oos.writeObject(checkpoint) - } { - oos.close() - } try { + val bytes = Checkpoint.serialize(checkpoint, conf) executor.execute(new CheckpointWriteHandler( - checkpoint.checkpointTime, bos.toByteArray, clearCheckpointDataLater)) + checkpoint.checkpointTime, bytes, clearCheckpointDataLater)) logDebug("Submitted checkpoint of time " + checkpoint.checkpointTime + " writer queue") } catch { case rej: RejectedExecutionException => @@ -264,25 +295,8 @@ object CheckpointReader extends Logging { checkpointFiles.foreach(file => { logInfo("Attempting to load checkpoint from file " + file) try { - var ois: ObjectInputStreamWithLoader = null - var cp: Checkpoint = null - Utils.tryWithSafeFinally { - val fis = fs.open(file) - // ObjectInputStream uses the last defined user-defined class loader in the stack - // to find classes, which maybe the wrong class loader. Hence, a inherited version - // of ObjectInputStream is used to explicitly use the current thread's default class - // loader to find and load classes. This is a well know Java issue and has popped up - // in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627) - val zis = compressionCodec.compressedInputStream(fis) - ois = new ObjectInputStreamWithLoader(zis, - Thread.currentThread().getContextClassLoader) - cp = ois.readObject.asInstanceOf[Checkpoint] - } { - if (ois != null) { - ois.close() - } - } - cp.validate() + val fis = fs.open(file) + val cp = Checkpoint.deserialize(fis, conf) logInfo("Checkpoint successfully loaded from file " + file) logInfo("Checkpoint was generated at time " + cp.checkpointTime) return Some(cp) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index fe614c4be590f..95063692e1146 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming -import java.io.InputStream +import java.io.{InputStream, NotSerializableException} import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} import scala.collection.Map @@ -35,6 +35,7 @@ import org.apache.spark._ import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.input.FixedLengthBinaryInputFormat import org.apache.spark.rdd.{RDD, RDDOperationScope} +import org.apache.spark.serializer.SerializationDebugger import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContextState._ import org.apache.spark.streaming.dstream._ @@ -235,6 +236,10 @@ class StreamingContext private[streaming] ( } } + private[streaming] def isCheckpointingEnabled: Boolean = { + checkpointDir != null + } + private[streaming] def initialCheckpoint: Checkpoint = { if (isCheckpointPresent) cp_ else null } @@ -523,11 +528,26 @@ class StreamingContext private[streaming] ( assert(graph != null, "Graph is null") graph.validate() - assert( - checkpointDir == null || checkpointDuration != null, + require( + !isCheckpointingEnabled || checkpointDuration != null, "Checkpoint directory has been set, but the graph checkpointing interval has " + "not been set. Please use StreamingContext.checkpoint() to set the interval." ) + + // Verify whether the DStream checkpoint is serializable + if (isCheckpointingEnabled) { + val checkpoint = new Checkpoint(this, Time.apply(0)) + try { + Checkpoint.serialize(checkpoint, conf) + } catch { + case e: NotSerializableException => + throw new NotSerializableException( + "DStream checkpointing has been enabled but the DStreams with their functions " + + "are not serializable\nSerialization stack:\n" + + SerializationDebugger.find(checkpoint).map("\t- " + _).mkString("\n") + ) + } + } } /** diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 4b12affbb0ddd..3a958bf3a3c19 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -17,21 +17,21 @@ package org.apache.spark.streaming -import java.io.File +import java.io.{File, NotSerializableException} import java.util.concurrent.atomic.AtomicInteger import org.apache.commons.io.FileUtils -import org.scalatest.{Assertions, BeforeAndAfter, FunSuite} -import org.scalatest.concurrent.Timeouts import org.scalatest.concurrent.Eventually._ +import org.scalatest.concurrent.Timeouts import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.SpanSugar._ +import org.scalatest.{Assertions, BeforeAndAfter, FunSuite} -import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.receiver.Receiver import org.apache.spark.util.Utils +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException} class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts with Logging { @@ -132,6 +132,25 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w } } + test("start with non-seriazable DStream checkpoints") { + val checkpointDir = Utils.createTempDir() + ssc = new StreamingContext(conf, batchDuration) + ssc.checkpoint(checkpointDir.getAbsolutePath) + addInputStream(ssc).foreachRDD { rdd => + // Refer to this.appName from inside closure so that this closure refers to + // the instance of StreamingContextSuite, and is therefore not serializable + rdd.count() + appName + } + + // Test whether start() fails early when checkpointing is enabled + val exception = intercept[NotSerializableException] { + ssc.start() + } + assert(exception.getMessage().contains("DStreams with their functions are not serializable")) + assert(ssc.getState() !== StreamingContextState.ACTIVE) + assert(StreamingContext.getActive().isEmpty) + } + test("start multiple times") { ssc = new StreamingContext(master, appName, batchDuration) addInputStream(ssc).register() From 7956dd7ab03e1542d89dd94c043f1e5131684199 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 20 May 2015 16:37:11 -0700 Subject: [PATCH 13/17] [SPARK-7698] Cache and reuse buffers in ExecutorMemoryAllocator when using heap allocation When on-heap memory allocation is used, ExecutorMemoryManager should maintain a cache / pool of buffers for re-use by tasks. This will significantly improve the performance of the new Tungsten's sort-shuffle for jobs with many short-lived tasks by eliminating a major source of GC. This pull request is a minimum-viable-implementation of this idea. In its current form, this patch significantly improves performance on a stress test which launches huge numbers of short-lived shuffle map tasks back-to-back in the same JVM. Author: Josh Rosen Closes #6227 from JoshRosen/SPARK-7698 and squashes the following commits: fd6cb55 [Josh Rosen] SoftReference -> WeakReference b154e86 [Josh Rosen] WIP sketch of pooling in ExecutorMemoryManager --- .../unsafe/memory/ExecutorMemoryManager.java | 57 ++++++++++++++++++- 1 file changed, 55 insertions(+), 2 deletions(-) diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/memory/ExecutorMemoryManager.java b/unsafe/src/main/java/org/apache/spark/unsafe/memory/ExecutorMemoryManager.java index 62c29c8cc1e4d..cbbe8594627a5 100644 --- a/unsafe/src/main/java/org/apache/spark/unsafe/memory/ExecutorMemoryManager.java +++ b/unsafe/src/main/java/org/apache/spark/unsafe/memory/ExecutorMemoryManager.java @@ -17,6 +17,12 @@ package org.apache.spark.unsafe.memory; +import java.lang.ref.WeakReference; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.Map; +import javax.annotation.concurrent.GuardedBy; + /** * Manages memory for an executor. Individual operators / tasks allocate memory through * {@link TaskMemoryManager} objects, which obtain their memory from ExecutorMemoryManager. @@ -33,6 +39,12 @@ public class ExecutorMemoryManager { */ final boolean inHeap; + @GuardedBy("this") + private final Map>> bufferPoolsBySize = + new HashMap>>(); + + private static final int POOLING_THRESHOLD_BYTES = 1024 * 1024; + /** * Construct a new ExecutorMemoryManager. * @@ -43,16 +55,57 @@ public ExecutorMemoryManager(MemoryAllocator allocator) { this.allocator = allocator; } + /** + * Returns true if allocations of the given size should go through the pooling mechanism and + * false otherwise. + */ + private boolean shouldPool(long size) { + // Very small allocations are less likely to benefit from pooling. + // At some point, we should explore supporting pooling for off-heap memory, but for now we'll + // ignore that case in the interest of simplicity. + return size >= POOLING_THRESHOLD_BYTES && allocator instanceof HeapMemoryAllocator; + } + /** * Allocates a contiguous block of memory. Note that the allocated memory is not guaranteed * to be zeroed out (call `zero()` on the result if this is necessary). */ MemoryBlock allocate(long size) throws OutOfMemoryError { - return allocator.allocate(size); + if (shouldPool(size)) { + synchronized (this) { + final LinkedList> pool = bufferPoolsBySize.get(size); + if (pool != null) { + while (!pool.isEmpty()) { + final WeakReference blockReference = pool.pop(); + final MemoryBlock memory = blockReference.get(); + if (memory != null) { + assert (memory.size() == size); + return memory; + } + } + bufferPoolsBySize.remove(size); + } + } + return allocator.allocate(size); + } else { + return allocator.allocate(size); + } } void free(MemoryBlock memory) { - allocator.free(memory); + final long size = memory.size(); + if (shouldPool(size)) { + synchronized (this) { + LinkedList> pool = bufferPoolsBySize.get(size); + if (pool == null) { + pool = new LinkedList>(); + bufferPoolsBySize.put(size, pool); + } + pool.add(new WeakReference(memory)); + } + } else { + allocator.free(memory); + } } } From f2faa7af30662e3bdf15780f8719c71108f8e30b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 20 May 2015 16:42:49 -0700 Subject: [PATCH 14/17] [SPARK-7251] Perform sequential scan when iterating over BytesToBytesMap This patch modifies `BytesToBytesMap.iterator()` to iterate through records in the order that they appear in the data pages rather than iterating through the hashtable pointer arrays. This results in fewer random memory accesses, significantly improving performance for scan-and-copy operations. This is possible because our data pages are laid out as sequences of `[keyLength][data][valueLength][data]` entries. In order to mark the end of a partially-filled data page, we write `-1` as a special end-of-page length (BytesToByesMap supports empty/zero-length keys and values, which is why we had to use a negative length). This patch incorporates / closes #5836. Author: Josh Rosen Closes #6159 from JoshRosen/SPARK-7251 and squashes the following commits: 05bd90a [Josh Rosen] Compare capacity, not size, to MAX_CAPACITY 2a20d71 [Josh Rosen] Fix maximum BytesToBytesMap capacity bc4854b [Josh Rosen] Guard against overflow when growing BytesToBytesMap f5feadf [Josh Rosen] Add test for iterating over an empty map 273b842 [Josh Rosen] [SPARK-7251] Perform sequential scan when iterating over entries in BytesToBytesMap --- unsafe/pom.xml | 5 + .../spark/unsafe/map/BytesToBytesMap.java | 151 ++++++++++++---- .../unsafe/map/HashMapGrowthStrategy.java | 4 +- .../unsafe/memory/TaskMemoryManager.java | 2 +- .../map/AbstractBytesToBytesMapSuite.java | 165 ++++++++++++++++-- 5 files changed, 274 insertions(+), 53 deletions(-) diff --git a/unsafe/pom.xml b/unsafe/pom.xml index 9e151fc7a9141..2fd17267ac427 100644 --- a/unsafe/pom.xml +++ b/unsafe/pom.xml @@ -65,6 +65,11 @@ junit-interface test + + org.mockito + mockito-all + test + target/scala-${scala.binary.version}/classes diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/unsafe/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 19d6a169fd2ad..bd4ca74cc7764 100644 --- a/unsafe/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/unsafe/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -23,6 +23,8 @@ import java.util.LinkedList; import java.util.List; +import com.google.common.annotations.VisibleForTesting; + import org.apache.spark.unsafe.*; import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.array.LongArray; @@ -36,9 +38,8 @@ * This is backed by a power-of-2-sized hash table, using quadratic probing with triangular numbers, * which is guaranteed to exhaust the space. *

- * The map can support up to 2^31 keys because we use 32 bit MurmurHash. If the key cardinality is - * higher than this, you should probably be using sorting instead of hashing for better cache - * locality. + * The map can support up to 2^29 keys. If the key cardinality is higher than this, you should + * probably be using sorting instead of hashing for better cache locality. *

* This class is not thread safe. */ @@ -48,6 +49,11 @@ public final class BytesToBytesMap { private static final HashMapGrowthStrategy growthStrategy = HashMapGrowthStrategy.DOUBLING; + /** + * Special record length that is placed after the last record in a data page. + */ + private static final int END_OF_PAGE_MARKER = -1; + private final TaskMemoryManager memoryManager; /** @@ -64,7 +70,7 @@ public final class BytesToBytesMap { /** * Offset into `currentDataPage` that points to the location where new data can be inserted into - * the page. + * the page. This does not incorporate the page's base offset. */ private long pageCursor = 0; @@ -74,6 +80,15 @@ public final class BytesToBytesMap { */ private static final long PAGE_SIZE_BYTES = 1L << 26; // 64 megabytes + /** + * The maximum number of keys that BytesToBytesMap supports. The hash table has to be + * power-of-2-sized and its backing Java array can contain at most (1 << 30) elements, since + * that's the largest power-of-2 that's less than Integer.MAX_VALUE. We need two long array + * entries per key, giving us a maximum capacity of (1 << 29). + */ + @VisibleForTesting + static final int MAX_CAPACITY = (1 << 29); + // This choice of page table size and page size means that we can address up to 500 gigabytes // of memory. @@ -143,6 +158,13 @@ public BytesToBytesMap( this.loadFactor = loadFactor; this.loc = new Location(); this.enablePerfMetrics = enablePerfMetrics; + if (initialCapacity <= 0) { + throw new IllegalArgumentException("Initial capacity must be greater than 0"); + } + if (initialCapacity > MAX_CAPACITY) { + throw new IllegalArgumentException( + "Initial capacity " + initialCapacity + " exceeds maximum capacity of " + MAX_CAPACITY); + } allocate(initialCapacity); } @@ -162,6 +184,55 @@ public BytesToBytesMap( */ public int size() { return size; } + private static final class BytesToBytesMapIterator implements Iterator { + + private final int numRecords; + private final Iterator dataPagesIterator; + private final Location loc; + + private int currentRecordNumber = 0; + private Object pageBaseObject; + private long offsetInPage; + + BytesToBytesMapIterator(int numRecords, Iterator dataPagesIterator, Location loc) { + this.numRecords = numRecords; + this.dataPagesIterator = dataPagesIterator; + this.loc = loc; + if (dataPagesIterator.hasNext()) { + advanceToNextPage(); + } + } + + private void advanceToNextPage() { + final MemoryBlock currentPage = dataPagesIterator.next(); + pageBaseObject = currentPage.getBaseObject(); + offsetInPage = currentPage.getBaseOffset(); + } + + @Override + public boolean hasNext() { + return currentRecordNumber != numRecords; + } + + @Override + public Location next() { + int keyLength = (int) PlatformDependent.UNSAFE.getLong(pageBaseObject, offsetInPage); + if (keyLength == END_OF_PAGE_MARKER) { + advanceToNextPage(); + keyLength = (int) PlatformDependent.UNSAFE.getLong(pageBaseObject, offsetInPage); + } + loc.with(pageBaseObject, offsetInPage); + offsetInPage += 8 + 8 + keyLength + loc.getValueLength(); + currentRecordNumber++; + return loc; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + } + /** * Returns an iterator for iterating over the entries of this map. * @@ -171,27 +242,7 @@ public BytesToBytesMap( * `lookup()`, the behavior of the returned iterator is undefined. */ public Iterator iterator() { - return new Iterator() { - - private int nextPos = bitset.nextSetBit(0); - - @Override - public boolean hasNext() { - return nextPos != -1; - } - - @Override - public Location next() { - final int pos = nextPos; - nextPos = bitset.nextSetBit(nextPos + 1); - return loc.with(pos, 0, true); - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - }; + return new BytesToBytesMapIterator(size, dataPages.iterator(), loc); } /** @@ -268,8 +319,11 @@ public final class Location { private int valueLength; private void updateAddressesAndSizes(long fullKeyAddress) { - final Object page = memoryManager.getPage(fullKeyAddress); - final long keyOffsetInPage = memoryManager.getOffsetInPage(fullKeyAddress); + updateAddressesAndSizes( + memoryManager.getPage(fullKeyAddress), memoryManager.getOffsetInPage(fullKeyAddress)); + } + + private void updateAddressesAndSizes(Object page, long keyOffsetInPage) { long position = keyOffsetInPage; keyLength = (int) PlatformDependent.UNSAFE.getLong(page, position); position += 8; // word used to store the key size @@ -291,6 +345,12 @@ Location with(int pos, int keyHashcode, boolean isDefined) { return this; } + Location with(Object page, long keyOffsetInPage) { + this.isDefined = true; + updateAddressesAndSizes(page, keyOffsetInPage); + return this; + } + /** * Returns true if the key is defined at this position, and false otherwise. */ @@ -345,6 +405,8 @@ public int getValueLength() { *

* It is only valid to call this method immediately after calling `lookup()` using the same key. *

+ * The key and value must be word-aligned (that is, their sizes must multiples of 8). + *

* After calling this method, calls to `get[Key|Value]Address()` and `get[Key|Value]Length` * will return information on the data stored by this `putNewKey` call. *

@@ -370,17 +432,27 @@ public void putNewKey( isDefined = true; assert (keyLengthBytes % 8 == 0); assert (valueLengthBytes % 8 == 0); + if (size == MAX_CAPACITY) { + throw new IllegalStateException("BytesToBytesMap has reached maximum capacity"); + } // Here, we'll copy the data into our data pages. Because we only store a relative offset from // the key address instead of storing the absolute address of the value, the key and value // must be stored in the same memory page. // (8 byte key length) (key) (8 byte value length) (value) final long requiredSize = 8 + keyLengthBytes + 8 + valueLengthBytes; - assert(requiredSize <= PAGE_SIZE_BYTES); + assert (requiredSize <= PAGE_SIZE_BYTES - 8); // Reserve 8 bytes for the end-of-page marker. size++; bitset.set(pos); - // If there's not enough space in the current page, allocate a new page: - if (currentDataPage == null || PAGE_SIZE_BYTES - pageCursor < requiredSize) { + // If there's not enough space in the current page, allocate a new page (8 bytes are reserved + // for the end-of-page marker). + if (currentDataPage == null || PAGE_SIZE_BYTES - 8 - pageCursor < requiredSize) { + if (currentDataPage != null) { + // There wasn't enough space in the current page, so write an end-of-page marker: + final Object pageBaseObject = currentDataPage.getBaseObject(); + final long lengthOffsetInPage = currentDataPage.getBaseOffset() + pageCursor; + PlatformDependent.UNSAFE.putLong(pageBaseObject, lengthOffsetInPage, END_OF_PAGE_MARKER); + } MemoryBlock newPage = memoryManager.allocatePage(PAGE_SIZE_BYTES); dataPages.add(newPage); pageCursor = 0; @@ -414,7 +486,7 @@ public void putNewKey( longArray.set(pos * 2 + 1, keyHashcode); updateAddressesAndSizes(storedKeyAddress); isDefined = true; - if (size > growthThreshold) { + if (size > growthThreshold && longArray.size() < MAX_CAPACITY) { growAndRehash(); } } @@ -427,8 +499,11 @@ public void putNewKey( * @param capacity the new map capacity */ private void allocate(int capacity) { - capacity = Math.max((int) Math.min(Integer.MAX_VALUE, nextPowerOf2(capacity)), 64); - longArray = new LongArray(memoryManager.allocate(capacity * 8 * 2)); + assert (capacity >= 0); + // The capacity needs to be divisible by 64 so that our bit set can be sized properly + capacity = Math.max((int) Math.min(MAX_CAPACITY, nextPowerOf2(capacity)), 64); + assert (capacity <= MAX_CAPACITY); + longArray = new LongArray(memoryManager.allocate(capacity * 8L * 2)); bitset = new BitSet(MemoryBlock.fromLongArray(new long[capacity / 64])); this.growthThreshold = (int) (capacity * loadFactor); @@ -494,10 +569,16 @@ public long getNumHashCollisions() { return numHashCollisions; } + @VisibleForTesting + int getNumDataPages() { + return dataPages.size(); + } + /** * Grows the size of the hash table and re-hash everything. */ - private void growAndRehash() { + @VisibleForTesting + void growAndRehash() { long resizeStartTime = -1; if (enablePerfMetrics) { resizeStartTime = System.nanoTime(); @@ -508,7 +589,7 @@ private void growAndRehash() { final int oldCapacity = (int) oldBitSet.capacity(); // Allocate the new data structures - allocate(Math.min(Integer.MAX_VALUE, growthStrategy.nextCapacity(oldCapacity))); + allocate(Math.min(growthStrategy.nextCapacity(oldCapacity), MAX_CAPACITY)); // Re-mask (we don't recompute the hashcode because we stored all 32 bits of it) for (int pos = oldBitSet.nextSetBit(0); pos >= 0; pos = oldBitSet.nextSetBit(pos + 1)) { diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/map/HashMapGrowthStrategy.java b/unsafe/src/main/java/org/apache/spark/unsafe/map/HashMapGrowthStrategy.java index 7c321baffe82d..20654e4eeaa02 100644 --- a/unsafe/src/main/java/org/apache/spark/unsafe/map/HashMapGrowthStrategy.java +++ b/unsafe/src/main/java/org/apache/spark/unsafe/map/HashMapGrowthStrategy.java @@ -32,7 +32,9 @@ public interface HashMapGrowthStrategy { class Doubling implements HashMapGrowthStrategy { @Override public int nextCapacity(int currentCapacity) { - return currentCapacity * 2; + assert (currentCapacity > 0); + // Guard against overflow + return (currentCapacity * 2 > 0) ? (currentCapacity * 2) : Integer.MAX_VALUE; } } diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/memory/TaskMemoryManager.java b/unsafe/src/main/java/org/apache/spark/unsafe/memory/TaskMemoryManager.java index 2906ac8abad1a..10881969dbc78 100644 --- a/unsafe/src/main/java/org/apache/spark/unsafe/memory/TaskMemoryManager.java +++ b/unsafe/src/main/java/org/apache/spark/unsafe/memory/TaskMemoryManager.java @@ -44,7 +44,7 @@ * maximum size of a long[] array, allowing us to address 8192 * 2^32 * 8 bytes, which is * approximately 35 terabytes of memory. */ -public final class TaskMemoryManager { +public class TaskMemoryManager { private final Logger logger = LoggerFactory.getLogger(TaskMemoryManager.class); diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/unsafe/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java index 7a5c0622d1ffb..81315f7c94645 100644 --- a/unsafe/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java +++ b/unsafe/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java @@ -25,24 +25,40 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import static org.mockito.AdditionalMatchers.geq; +import static org.mockito.Mockito.*; import org.apache.spark.unsafe.array.ByteArrayMethods; +import org.apache.spark.unsafe.memory.*; import org.apache.spark.unsafe.PlatformDependent; import static org.apache.spark.unsafe.PlatformDependent.BYTE_ARRAY_OFFSET; -import org.apache.spark.unsafe.memory.ExecutorMemoryManager; -import org.apache.spark.unsafe.memory.MemoryAllocator; -import org.apache.spark.unsafe.memory.MemoryLocation; -import org.apache.spark.unsafe.memory.TaskMemoryManager; +import static org.apache.spark.unsafe.PlatformDependent.LONG_ARRAY_OFFSET; + public abstract class AbstractBytesToBytesMapSuite { private final Random rand = new Random(42); private TaskMemoryManager memoryManager; + private TaskMemoryManager sizeLimitedMemoryManager; @Before public void setup() { memoryManager = new TaskMemoryManager(new ExecutorMemoryManager(getMemoryAllocator())); + // Mocked memory manager for tests that check the maximum array size, since actually allocating + // such large arrays will cause us to run out of memory in our tests. + sizeLimitedMemoryManager = spy(memoryManager); + when(sizeLimitedMemoryManager.allocate(geq(1L << 20))).thenAnswer(new Answer() { + @Override + public MemoryBlock answer(InvocationOnMock invocation) throws Throwable { + if (((Long) invocation.getArguments()[0] / 8) > Integer.MAX_VALUE) { + throw new OutOfMemoryError("Requested array size exceeds VM limit"); + } + return memoryManager.allocate(1L << 20); + } + }); } @After @@ -101,6 +117,7 @@ public void emptyMap() { final int keyLengthInBytes = keyLengthInWords * 8; final byte[] key = getRandomByteArray(keyLengthInWords); Assert.assertFalse(map.lookup(key, BYTE_ARRAY_OFFSET, keyLengthInBytes).isDefined()); + Assert.assertFalse(map.iterator().hasNext()); } finally { map.free(); } @@ -159,7 +176,7 @@ public void setAndRetrieveAKey() { @Test public void iteratorTest() throws Exception { - final int size = 128; + final int size = 4096; BytesToBytesMap map = new BytesToBytesMap(memoryManager, size / 2); try { for (long i = 0; i < size; i++) { @@ -167,14 +184,26 @@ public void iteratorTest() throws Exception { final BytesToBytesMap.Location loc = map.lookup(value, PlatformDependent.LONG_ARRAY_OFFSET, 8); Assert.assertFalse(loc.isDefined()); - loc.putNewKey( - value, - PlatformDependent.LONG_ARRAY_OFFSET, - 8, - value, - PlatformDependent.LONG_ARRAY_OFFSET, - 8 - ); + // Ensure that we store some zero-length keys + if (i % 5 == 0) { + loc.putNewKey( + null, + PlatformDependent.LONG_ARRAY_OFFSET, + 0, + value, + PlatformDependent.LONG_ARRAY_OFFSET, + 8 + ); + } else { + loc.putNewKey( + value, + PlatformDependent.LONG_ARRAY_OFFSET, + 8, + value, + PlatformDependent.LONG_ARRAY_OFFSET, + 8 + ); + } } final java.util.BitSet valuesSeen = new java.util.BitSet(size); final Iterator iter = map.iterator(); @@ -183,11 +212,16 @@ public void iteratorTest() throws Exception { Assert.assertTrue(loc.isDefined()); final MemoryLocation keyAddress = loc.getKeyAddress(); final MemoryLocation valueAddress = loc.getValueAddress(); - final long key = PlatformDependent.UNSAFE.getLong( - keyAddress.getBaseObject(), keyAddress.getBaseOffset()); final long value = PlatformDependent.UNSAFE.getLong( valueAddress.getBaseObject(), valueAddress.getBaseOffset()); - Assert.assertEquals(key, value); + final long keyLength = loc.getKeyLength(); + if (keyLength == 0) { + Assert.assertTrue("value " + value + " was not divisible by 5", value % 5 == 0); + } else { + final long key = PlatformDependent.UNSAFE.getLong( + keyAddress.getBaseObject(), keyAddress.getBaseOffset()); + Assert.assertEquals(value, key); + } valuesSeen.set((int) value); } Assert.assertEquals(size, valuesSeen.cardinality()); @@ -196,6 +230,74 @@ public void iteratorTest() throws Exception { } } + @Test + public void iteratingOverDataPagesWithWastedSpace() throws Exception { + final int NUM_ENTRIES = 1000 * 1000; + final int KEY_LENGTH = 16; + final int VALUE_LENGTH = 40; + final BytesToBytesMap map = new BytesToBytesMap(memoryManager, NUM_ENTRIES); + // Each record will take 8 + 8 + 16 + 40 = 72 bytes of space in the data page. Our 64-megabyte + // pages won't be evenly-divisible by records of this size, which will cause us to waste some + // space at the end of the page. This is necessary in order for us to take the end-of-record + // handling branch in iterator(). + try { + for (int i = 0; i < NUM_ENTRIES; i++) { + final long[] key = new long[] { i, i }; // 2 * 8 = 16 bytes + final long[] value = new long[] { i, i, i, i, i }; // 5 * 8 = 40 bytes + final BytesToBytesMap.Location loc = map.lookup( + key, + LONG_ARRAY_OFFSET, + KEY_LENGTH + ); + Assert.assertFalse(loc.isDefined()); + loc.putNewKey( + key, + LONG_ARRAY_OFFSET, + KEY_LENGTH, + value, + LONG_ARRAY_OFFSET, + VALUE_LENGTH + ); + } + Assert.assertEquals(2, map.getNumDataPages()); + + final java.util.BitSet valuesSeen = new java.util.BitSet(NUM_ENTRIES); + final Iterator iter = map.iterator(); + final long key[] = new long[KEY_LENGTH / 8]; + final long value[] = new long[VALUE_LENGTH / 8]; + while (iter.hasNext()) { + final BytesToBytesMap.Location loc = iter.next(); + Assert.assertTrue(loc.isDefined()); + Assert.assertEquals(KEY_LENGTH, loc.getKeyLength()); + Assert.assertEquals(VALUE_LENGTH, loc.getValueLength()); + PlatformDependent.copyMemory( + loc.getKeyAddress().getBaseObject(), + loc.getKeyAddress().getBaseOffset(), + key, + LONG_ARRAY_OFFSET, + KEY_LENGTH + ); + PlatformDependent.copyMemory( + loc.getValueAddress().getBaseObject(), + loc.getValueAddress().getBaseOffset(), + value, + LONG_ARRAY_OFFSET, + VALUE_LENGTH + ); + for (long j : key) { + Assert.assertEquals(key[0], j); + } + for (long j : value) { + Assert.assertEquals(key[0], j); + } + valuesSeen.set((int) key[0]); + } + Assert.assertEquals(NUM_ENTRIES, valuesSeen.cardinality()); + } finally { + map.free(); + } + } + @Test public void randomizedStressTest() { final int size = 65536; @@ -247,4 +349,35 @@ public void randomizedStressTest() { map.free(); } } + + @Test + public void initialCapacityBoundsChecking() { + try { + new BytesToBytesMap(sizeLimitedMemoryManager, 0); + Assert.fail("Expected IllegalArgumentException to be thrown"); + } catch (IllegalArgumentException e) { + // expected exception + } + + try { + new BytesToBytesMap(sizeLimitedMemoryManager, BytesToBytesMap.MAX_CAPACITY + 1); + Assert.fail("Expected IllegalArgumentException to be thrown"); + } catch (IllegalArgumentException e) { + // expected exception + } + + // Can allocate _at_ the max capacity + BytesToBytesMap map = + new BytesToBytesMap(sizeLimitedMemoryManager, BytesToBytesMap.MAX_CAPACITY); + map.free(); + } + + @Test + public void resizingLargeMap() { + // As long as a map's capacity is below the max, we should be able to resize up to the max + BytesToBytesMap map = + new BytesToBytesMap(sizeLimitedMemoryManager, BytesToBytesMap.MAX_CAPACITY - 64); + map.growAndRehash(); + map.free(); + } } From c330e52dae6a3ec7e67ca82e2c2f4ea873976458 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 20 May 2015 17:26:26 -0700 Subject: [PATCH 15/17] [SPARK-7762] [MLLIB] set default value for outputCol Set a default value for `outputCol` instead of forcing users to name it. This is useful for intermediate transformers in the pipeline. jkbradley Author: Xiangrui Meng Closes #6289 from mengxr/SPARK-7762 and squashes the following commits: 54edebc [Xiangrui Meng] merge master bff8667 [Xiangrui Meng] update unit test 171246b [Xiangrui Meng] add unit test for outputCol a4321bd [Xiangrui Meng] set default value for outputCol --- .../ml/param/shared/SharedParamsCodeGen.scala | 2 +- .../spark/ml/param/shared/sharedParams.scala | 4 ++- .../ml/param/shared/SharedParamsSuite.scala | 35 +++++++++++++++++++ .../ml/param/_shared_params_code_gen.py | 2 +- python/pyspark/ml/param/shared.py | 3 +- 5 files changed, 42 insertions(+), 4 deletions(-) create mode 100644 mllib/src/test/scala/org/apache/spark/ml/param/shared/SharedParamsSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala index 8b8cb81373a65..1ffb5eddc36bd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -49,7 +49,7 @@ private[shared] object SharedParamsCodeGen { isValid = "ParamValidators.inRange(0, 1)"), ParamDesc[String]("inputCol", "input column name"), ParamDesc[Array[String]]("inputCols", "input column names"), - ParamDesc[String]("outputCol", "output column name"), + ParamDesc[String]("outputCol", "output column name", Some("uid + \"__output\"")), ParamDesc[Int]("checkpointInterval", "checkpoint interval (>= 1)", isValid = "ParamValidators.gtEq(1)"), ParamDesc[Boolean]("fitIntercept", "whether to fit an intercept term", Some("true")), diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala index 3a4976d3ddcd1..ed08417bd4df8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -185,7 +185,7 @@ private[ml] trait HasInputCols extends Params { } /** - * (private[ml]) Trait for shared param outputCol. + * (private[ml]) Trait for shared param outputCol (default: uid + "__output"). */ private[ml] trait HasOutputCol extends Params { @@ -195,6 +195,8 @@ private[ml] trait HasOutputCol extends Params { */ final val outputCol: Param[String] = new Param[String](this, "outputCol", "output column name") + setDefault(outputCol, uid + "__output") + /** @group getParam */ final def getOutputCol: String = $(outputCol) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/shared/SharedParamsSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/param/shared/SharedParamsSuite.scala new file mode 100644 index 0000000000000..ca18fa1ad3c15 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/param/shared/SharedParamsSuite.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.param.shared + +import org.scalatest.FunSuite + +import org.apache.spark.ml.param.Params + +class SharedParamsSuite extends FunSuite { + + test("outputCol") { + + class Obj(override val uid: String) extends Params with HasOutputCol + + val obj = new Obj("obj") + + assert(obj.hasDefault(obj.outputCol)) + assert(obj.getOrDefault(obj.outputCol) === "obj__output") + } +} diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index ccb929af184b8..69efc424ec4ef 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -116,7 +116,7 @@ def get$Name(self): ("rawPredictionCol", "raw prediction (a.k.a. confidence) column name", "'rawPrediction'"), ("inputCol", "input column name", None), ("inputCols", "input column names", None), - ("outputCol", "output column name", None), + ("outputCol", "output column name", "self.uid + '__output'"), ("numFeatures", "number of features", None), ("checkpointInterval", "checkpoint interval (>= 1)", None), ("seed", "random seed", "hash(type(self).__name__)"), diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 0b93788899124..bc088e4c29e26 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -280,6 +280,7 @@ def __init__(self): super(HasOutputCol, self).__init__() #: param for output column name self.outputCol = Param(self, "outputCol", "output column name") + self._setDefault(outputCol=self.uid + '__output') def setOutputCol(self, value): """ @@ -459,7 +460,7 @@ def __init__(self): self.maxMemoryInMB = Param(self, "maxMemoryInMB", "Maximum memory in MB allocated to histogram aggregation.") #: param for If false, the algorithm will pass trees to executors to match instances with nodes. If true, the algorithm will cache node IDs for each instance. Caching can speed up training of deeper trees. self.cacheNodeIds = Param(self, "cacheNodeIds", "If false, the algorithm will pass trees to executors to match instances with nodes. If true, the algorithm will cache node IDs for each instance. Caching can speed up training of deeper trees.") - + def setMaxDepth(self, value): """ Sets the value of :py:attr:`maxDepth`. From 5196efff53af4965ff216a9d5c0f8b2b4fc98652 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 20 May 2015 17:52:50 -0700 Subject: [PATCH 16/17] [SPARK-7719] Re-add UnsafeShuffleWriterSuite test that was removed for Java 6 compat This patch re-adds a test which was removed in 9ebb44f8abb1a13f045eed60190954db904ffef7 due to a Java 6 compatibility issue. We now use Guava's `Iterators.emptyIterator()` in place of `Collections.emptyIterator()`, which isn't present in all Java 6 versions. Author: Josh Rosen Closes #6298 from JoshRosen/SPARK-7719-fix-java-6-test-code and squashes the following commits: 5c9bd85 [Josh Rosen] Re-add UnsafeShuffleWriterSuite.emptyIterator() test which was removed due to Java 6 issue --- .../shuffle/unsafe/UnsafeShuffleWriterSuite.java | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java index 03116d8fc2b21..83d109115aa5c 100644 --- a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java @@ -26,6 +26,7 @@ import scala.reflect.ClassTag; import scala.runtime.AbstractFunction1; +import com.google.common.collect.Iterators; import com.google.common.collect.HashMultiset; import com.google.common.io.ByteStreams; import org.junit.After; @@ -252,6 +253,20 @@ public void doNotNeedToCallWriteBeforeUnsuccessfulStop() throws IOException { createWriter(false).stop(false); } + @Test + public void writeEmptyIterator() throws Exception { + final UnsafeShuffleWriter writer = createWriter(true); + writer.write(Iterators.>emptyIterator()); + final Option mapStatus = writer.stop(true); + assertTrue(mapStatus.isDefined()); + assertTrue(mergedOutputFile.exists()); + assertArrayEquals(new long[NUM_PARTITITONS], partitionSizesInMergedFile); + assertEquals(0, taskMetrics.shuffleWriteMetrics().get().shuffleRecordsWritten()); + assertEquals(0, taskMetrics.shuffleWriteMetrics().get().shuffleBytesWritten()); + assertEquals(0, taskMetrics.diskBytesSpilled()); + assertEquals(0, taskMetrics.memoryBytesSpilled()); + } + @Test public void writeWithoutSpilling() throws Exception { // In this example, each partition should have exactly one record: From a70bf06b790add5f279a69607df89ed36155b0e4 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 20 May 2015 21:13:10 -0500 Subject: [PATCH 17/17] =?UTF-8?q?[SPARK-7750]=20[WEBUI]=20Rename=20endpoin?= =?UTF-8?q?ts=20from=20`json`=20to=20`api`=20to=20allow=20fu=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …rther extension to non-json outputs too. Author: Hari Shreedharan Closes #6273 from harishreedharan/json-to-api and squashes the following commits: e14b73b [Hari Shreedharan] Rename `getJsonServlet` to `getServletHandler` i 42f8acb [Hari Shreedharan] Import order fixes. 2ef852f [Hari Shreedharan] [SPARK-7750][WebUI] Rename endpoints from `json` to `api` to allow further extension to non-json outputs too. --- .../spark/deploy/history/HistoryServer.scala | 5 +++-- .../spark/deploy/master/ui/MasterWebUI.scala | 5 +++-- ...nRootResource.scala => ApiRootResource.scala} | 8 ++++---- .../main/scala/org/apache/spark/ui/SparkUI.scala | 5 +++-- .../deploy/history/HistoryServerSuite.scala | 4 ++-- .../org/apache/spark/ui/UISeleniumSuite.scala | 16 ++++++++-------- docs/monitoring.md | 10 +++++----- 7 files changed, 28 insertions(+), 25 deletions(-) rename core/src/main/scala/org/apache/spark/status/api/v1/{JsonRootResource.scala => ApiRootResource.scala} (97%) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 517cbe5176241..5a0eb585a9049 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -25,7 +25,8 @@ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.status.api.v1.{ApplicationInfo, ApplicationsListResource, JsonRootResource, UIRoot} +import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationInfo, ApplicationsListResource, + UIRoot} import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{SignalLogger, Utils} @@ -125,7 +126,7 @@ class HistoryServer( def initialize() { attachPage(new HistoryPage(this)) - attachHandler(JsonRootResource.getJsonServlet(this)) + attachHandler(ApiRootResource.getServletHandler(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index eb26e9f99c70b..2111a8581f2e4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -19,7 +19,8 @@ package org.apache.spark.deploy.master.ui import org.apache.spark.Logging import org.apache.spark.deploy.master.Master -import org.apache.spark.status.api.v1.{ApplicationsListResource, ApplicationInfo, JsonRootResource, UIRoot} +import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationsListResource, ApplicationInfo, + UIRoot} import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.RpcUtils @@ -47,7 +48,7 @@ class MasterWebUI(val master: Master, requestedPort: Int) attachPage(new HistoryNotFoundPage(this)) attachPage(masterPage) attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) - attachHandler(JsonRootResource.getJsonServlet(this)) + attachHandler(ApiRootResource.getServletHandler(this)) attachHandler(createRedirectHandler( "/app/kill", "/", masterPage.handleAppKillRequest, httpMethods = Set("POST"))) attachHandler(createRedirectHandler( diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala rename to core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index c3ec45f54681b..bf2cc2e72f1fe 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -39,7 +39,7 @@ import org.apache.spark.ui.SparkUI * HistoryServerSuite. */ @Path("/v1") -private[v1] class JsonRootResource extends UIRootFromServletContext { +private[v1] class ApiRootResource extends UIRootFromServletContext { @Path("applications") def getApplicationList(): ApplicationListResource = { @@ -166,11 +166,11 @@ private[v1] class JsonRootResource extends UIRootFromServletContext { } -private[spark] object JsonRootResource { +private[spark] object ApiRootResource { - def getJsonServlet(uiRoot: UIRoot): ServletContextHandler = { + def getServletHandler(uiRoot: UIRoot): ServletContextHandler = { val jerseyContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS) - jerseyContext.setContextPath("/json") + jerseyContext.setContextPath("/api") val holder:ServletHolder = new ServletHolder(classOf[ServletContainer]) holder.setInitParameter("com.sun.jersey.config.property.resourceConfigClass", "com.sun.jersey.api.core.PackagesResourceConfig") diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index bfe4a180e8a6f..0b11e914bb251 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -19,7 +19,8 @@ package org.apache.spark.ui import java.util.Date -import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo, JsonRootResource, UIRoot} +import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationAttemptInfo, ApplicationInfo, + UIRoot} import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener @@ -64,7 +65,7 @@ private[spark] class SparkUI private ( attachTab(new ExecutorsTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath)) - attachHandler(JsonRootResource.getJsonServlet(this)) + attachHandler(ApiRootResource.getServletHandler(this)) // This should be POST only, but, the YARN AM proxy won't proxy POSTs attachHandler(createRedirectHandler( "/stages/stage/kill", "/stages", stagesTab.handleKillRequest, diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 318ab5dbc4804..4adb5122bcf1a 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -198,11 +198,11 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with } def getContentAndCode(path: String, port: Int = port): (Int, Option[String], Option[String]) = { - HistoryServerSuite.getContentAndCode(new URL(s"http://localhost:$port/json/v1/$path")) + HistoryServerSuite.getContentAndCode(new URL(s"http://localhost:$port/api/v1/$path")) } def getUrl(path: String): String = { - HistoryServerSuite.getUrl(new URL(s"http://localhost:$port/json/v1/$path")) + HistoryServerSuite.getUrl(new URL(s"http://localhost:$port/api/v1/$path")) } def generateExpectation(name: String, path: String): Unit = { diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 117b2c3960820..b6f5accef0cef 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -497,7 +497,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before goToUi(sc, "/jobs/job/?id=7") find("no-info").get.text should be ("No information to display for job 7") - val badJob = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get, "jobs/7")) + val badJob = HistoryServerSuite.getContentAndCode(apiUrl(sc.ui.get, "jobs/7")) badJob._1 should be (HttpServletResponse.SC_NOT_FOUND) badJob._2 should be (None) badJob._3 should be (Some("unknown job: 7")) @@ -540,18 +540,18 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before goToUi(sc, "/stages/stage/?id=12&attempt=0") find("no-info").get.text should be ("No information to display for Stage 12 (Attempt 0)") - val badStage = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get,"stages/12/0")) + val badStage = HistoryServerSuite.getContentAndCode(apiUrl(sc.ui.get,"stages/12/0")) badStage._1 should be (HttpServletResponse.SC_NOT_FOUND) badStage._2 should be (None) badStage._3 should be (Some("unknown stage: 12")) - val badAttempt = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get,"stages/19/15")) + val badAttempt = HistoryServerSuite.getContentAndCode(apiUrl(sc.ui.get,"stages/19/15")) badAttempt._1 should be (HttpServletResponse.SC_NOT_FOUND) badAttempt._2 should be (None) badAttempt._3 should be (Some("unknown attempt for stage 19. Found attempts: [0]")) val badStageAttemptList = HistoryServerSuite.getContentAndCode( - jsonUrl(sc.ui.get, "stages/12")) + apiUrl(sc.ui.get, "stages/12")) badStageAttemptList._1 should be (HttpServletResponse.SC_NOT_FOUND) badStageAttemptList._2 should be (None) badStageAttemptList._3 should be (Some("unknown stage: 12")) @@ -561,7 +561,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before test("live UI json application list") { withSpark(newSparkContext()) { sc => val appListRawJson = HistoryServerSuite.getUrl(new URL( - sc.ui.get.appUIAddress + "/json/v1/applications")) + sc.ui.get.appUIAddress + "/api/v1/applications")) val appListJsonAst = JsonMethods.parse(appListRawJson) appListJsonAst.children.length should be (1) val attempts = (appListJsonAst \ "attempts").children @@ -587,10 +587,10 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } def getJson(ui: SparkUI, path: String): JValue = { - JsonMethods.parse(HistoryServerSuite.getUrl(jsonUrl(ui, path))) + JsonMethods.parse(HistoryServerSuite.getUrl(apiUrl(ui, path))) } - def jsonUrl(ui: SparkUI, path: String): URL = { - new URL(ui.appUIAddress + "/json/v1/applications/test/" + path) + def apiUrl(ui: SparkUI, path: String): URL = { + new URL(ui.appUIAddress + "/api/v1/applications/test/" + path) } } diff --git a/docs/monitoring.md b/docs/monitoring.md index 1e0fc150862fb..e75018499003a 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -178,9 +178,9 @@ Note that the history server only displays completed Spark jobs. One way to sign In addition to viewing the metrics in the UI, they are also available as JSON. This gives developers an easy way to create new visualizations and monitoring tools for Spark. The JSON is available for -both running applications, and in the history server. The endpoints are mounted at `/json/v1`. Eg., -for the history server, they would typically be accessible at `http://:18080/json/v1`, and -for a running application, at `http://localhost:4040/json/v1`. +both running applications, and in the history server. The endpoints are mounted at `/api/v1`. Eg., +for the history server, they would typically be accessible at `http://:18080/api/v1`, and +for a running application, at `http://localhost:4040/api/v1`. @@ -240,12 +240,12 @@ These endpoints have been strongly versioned to make it easier to develop applic * Individual fields will never be removed for any given endpoint * New endpoints may be added * New fields may be added to existing endpoints -* New versions of the api may be added in the future at a separate endpoint (eg., `json/v2`). New versions are *not* required to be backwards compatible. +* New versions of the api may be added in the future at a separate endpoint (eg., `api/v2`). New versions are *not* required to be backwards compatible. * Api versions may be dropped, but only after at least one minor release of co-existing with a new api version Note that even when examining the UI of a running applications, the `applications/[app-id]` portion is still required, though there is only one application available. Eg. to see the list of jobs for the -running app, you would go to `http://localhost:4040/json/v1/applications/[app-id]/jobs`. This is to +running app, you would go to `http://localhost:4040/api/v1/applications/[app-id]/jobs`. This is to keep the paths consistent in both modes. # Metrics
EndpointMeaning